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 17:50:57 UTC

[1/2] nifi-registry git commit: NIFIREG-52 Add Kerberos Support

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


http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosSpnegoFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosSpnegoFactory.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosSpnegoFactory.java
new file mode 100644
index 0000000..66d369d
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosSpnegoFactory.java
@@ -0,0 +1,67 @@
+/*
+ * 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.kerberos;
+
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.security.kerberos.authentication.KerberosServiceAuthenticationProvider;
+import org.springframework.security.kerberos.authentication.KerberosTicketValidator;
+
+@Configuration
+public class KerberosSpnegoFactory {
+
+    @Autowired
+    private NiFiRegistryProperties properties;
+
+    @Autowired(required = false)
+    private KerberosTicketValidator kerberosTicketValidator;
+
+    private KerberosServiceAuthenticationProvider kerberosServiceAuthenticationProvider;
+    private KerberosSpnegoIdentityProvider kerberosSpnegoIdentityProvider;
+
+    @Bean
+    public KerberosSpnegoIdentityProvider kerberosSpnegoIdentityProvider() throws Exception {
+
+        if (kerberosSpnegoIdentityProvider == null && properties.isKerberosSpnegoSupportEnabled()) {
+            kerberosSpnegoIdentityProvider = new KerberosSpnegoIdentityProvider(
+                    kerberosServiceAuthenticationProvider(),
+                    properties);
+        }
+
+        return kerberosSpnegoIdentityProvider;
+    }
+
+    @Bean
+    public KerberosServiceAuthenticationProvider kerberosServiceAuthenticationProvider() throws Exception {
+
+        if (kerberosServiceAuthenticationProvider == null && properties.isKerberosSpnegoSupportEnabled()) {
+
+            KerberosServiceAuthenticationProvider ksap = new KerberosServiceAuthenticationProvider();
+            ksap.setTicketValidator(kerberosTicketValidator);
+            ksap.setUserDetailsService(new KerberosUserDetailsService());
+            ksap.afterPropertiesSet();
+
+            kerberosServiceAuthenticationProvider = ksap;
+
+        }
+
+        return kerberosServiceAuthenticationProvider;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosSpnegoIdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosSpnegoIdentityProvider.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosSpnegoIdentityProvider.java
new file mode 100644
index 0000000..7002792
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosSpnegoIdentityProvider.java
@@ -0,0 +1,177 @@
+/*
+ * 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.kerberos;
+
+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.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.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 org.apache.nifi.registry.security.util.CryptoUtils;
+import org.apache.nifi.registry.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.authentication.AuthenticationDetailsSource;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.AuthenticationException;
+import org.springframework.security.crypto.codec.Base64;
+import org.springframework.security.kerberos.authentication.KerberosServiceAuthenticationProvider;
+import org.springframework.security.kerberos.authentication.KerberosServiceRequestToken;
+import org.springframework.security.web.authentication.WebAuthenticationDetailsSource;
+import org.springframework.stereotype.Component;
+
+import javax.servlet.http.HttpServletRequest;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.TimeUnit;
+
+@Component
+public class KerberosSpnegoIdentityProvider implements IdentityProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(KerberosSpnegoIdentityProvider.class);
+
+    private static final String issuer = KerberosSpnegoIdentityProvider.class.getSimpleName();
+
+    private static final IdentityProviderUsage usage = new IdentityProviderUsage() {
+        @Override
+        public String getText() {
+            return "The Kerberos user credentials must be passed in the HTTP Authorization header as specified by SPNEGO-based Kerberos. " +
+                    "That is: 'Authorization: Negotiate <kerberosTicket>', " +
+                    "where <kerberosTicket> is a value that will be validated by this identity provider against a Kerberos cluster.";
+        }
+
+        @Override
+        public AuthType getAuthType() {
+            return AuthType.NEGOTIATE;
+        }
+    };
+
+    private static final String AUTHORIZATION = "Authorization";
+    private static final String AUTHORIZATION_NEGOTIATE = "Negotiate";
+
+    private long expiration = TimeUnit.MILLISECONDS.convert(12, TimeUnit.HOURS);;
+    private KerberosServiceAuthenticationProvider kerberosServiceAuthenticationProvider;
+    private AuthenticationDetailsSource<HttpServletRequest, ?> authenticationDetailsSource;
+
+    @Autowired
+    public KerberosSpnegoIdentityProvider(KerberosServiceAuthenticationProvider kerberosServiceAuthenticationProvider, NiFiRegistryProperties properties) {
+        this.kerberosServiceAuthenticationProvider = kerberosServiceAuthenticationProvider;
+        authenticationDetailsSource = new WebAuthenticationDetailsSource();
+
+        final String expirationFromProperties = properties.getKerberosSpnegoAuthenticationExpiration();
+        if (expirationFromProperties != null) {
+            long expiration = FormatUtils.getTimeDuration(expirationFromProperties, TimeUnit.MILLISECONDS);
+        }
+    }
+
+    @Override
+    public IdentityProviderUsage getUsageInstructions() {
+        return usage;
+    }
+
+    @Override
+    public AuthenticationRequest extractCredentials(HttpServletRequest request) {
+
+        // Only support Kerberos authentication when running securely
+        if (!request.isSecure()) {
+            return null;
+        }
+
+        String headerValue = request.getHeader(AUTHORIZATION);
+
+        if (!isValidKerberosHeader(headerValue)) {
+            return null;
+        }
+
+        logger.debug("Detected 'Authorization: Negotiate header in request {}", request.getRequestURL());
+        byte[] base64Token = headerValue.substring(headerValue.indexOf(" ") + 1).getBytes(StandardCharsets.UTF_8);
+        byte[] kerberosTicket = Base64.decode(base64Token);
+        if (kerberosTicket != null) {
+            logger.debug("Successfully decoded SPNEGO/Kerberos ticket passed in Authorization: Negotiate <ticket> header.", request.getRequestURL());
+        }
+
+        return new AuthenticationRequest(null, kerberosTicket, authenticationDetailsSource.buildDetails(request));
+
+    }
+
+    @Override
+    public AuthenticationResponse authenticate(AuthenticationRequest authenticationRequest) throws InvalidCredentialsException, IdentityAccessException {
+
+        if (authenticationRequest == null) {
+            logger.info("Cannot authenticate null authenticationRequest, returning null.");
+            return null;
+        }
+
+        final Object credentials = authenticationRequest.getCredentials();
+        byte[] kerberosTicket = credentials != null && credentials instanceof byte[] ? (byte[]) authenticationRequest.getCredentials() : null;
+
+        if (credentials == null) {
+            logger.info("Kerberos Ticket not found in authenticationRequest credentials, returning null.");
+            return null;
+        }
+
+        try {
+            KerberosServiceRequestToken kerberosServiceRequestToken = new KerberosServiceRequestToken(kerberosTicket);
+            kerberosServiceRequestToken.setDetails(authenticationRequest.getDetails());
+            Authentication authentication = kerberosServiceAuthenticationProvider.authenticate(kerberosServiceRequestToken);
+            if (authentication == null) {
+                throw new InvalidCredentialsException("Kerberos credentials could not be authenticated.");
+            }
+
+            final String kerberosPrincipal = authentication.getName();
+
+            return new AuthenticationResponse(kerberosPrincipal, kerberosPrincipal, expiration, issuer);
+
+        } catch (AuthenticationException e) {
+            String authFailedMessage = "Kerberos credentials could not be authenticated.";
+
+            /* Kerberos uses encryption with up to AES-256, specifically AES256-CTS-HMAC-SHA1-96.
+             * That is not available in every JRE, particularly if Unlimited Strength Encryption
+             * policies are not installed in the Java home lib dir. The Kerberos lib does not
+             * differentiate between failures due to decryption and those due to bad credentials
+             * without walking the causes of the exception, so this check puts something
+             * potentially useful in the logs for those troubleshooting Kerberos authentication. */
+            if (!Boolean.FALSE.equals(CryptoUtils.isCryptoRestricted())) {
+                authFailedMessage += " This Java Runtime does not support unlimited strength encryption. " +
+                        "This could cause Kerberos authentication to fail as it can require AES-256.";
+            }
+
+            logger.info(authFailedMessage);
+            throw new InvalidCredentialsException(authFailedMessage, e);
+        }
+
+    }
+
+    @Override
+    public void onConfigured(IdentityProviderConfigurationContext configurationContext) throws SecurityProviderCreationException {
+        throw new SecurityProviderCreationException(KerberosSpnegoIdentityProvider.class.getSimpleName() +
+                " does not currently support being loaded via IdentityProviderFactory");
+    }
+
+    @Override
+    public void preDestruction() throws SecurityProviderDestructionException {
+    }
+
+    public boolean isValidKerberosHeader(String headerValue) {
+        return headerValue != null && (headerValue.startsWith(AUTHORIZATION_NEGOTIATE + " ") || headerValue.startsWith("Kerberos "));
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosTicketValidatorFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosTicketValidatorFactory.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosTicketValidatorFactory.java
new file mode 100644
index 0000000..ed3e6eb
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosTicketValidatorFactory.java
@@ -0,0 +1,69 @@
+/*
+ * 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.kerberos;
+
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.core.io.FileSystemResource;
+import org.springframework.security.kerberos.authentication.KerberosTicketValidator;
+import org.springframework.security.kerberos.authentication.sun.GlobalSunJaasKerberosConfig;
+import org.springframework.security.kerberos.authentication.sun.SunJaasKerberosTicketValidator;
+
+import java.io.File;
+
+@Configuration
+public class KerberosTicketValidatorFactory {
+
+    private NiFiRegistryProperties properties;
+
+    private KerberosTicketValidator kerberosTicketValidator;
+
+    @Autowired
+    public KerberosTicketValidatorFactory(NiFiRegistryProperties properties) {
+        this.properties = properties;
+    }
+
+    @Bean
+    public KerberosTicketValidator kerberosTicketValidator() throws Exception {
+
+        if (kerberosTicketValidator == null && properties.isKerberosSpnegoSupportEnabled()) {
+
+            // Configure SunJaasKerberos (global)
+            final File krb5ConfigFile = properties.getKerberosConfigurationFile();
+            if (krb5ConfigFile != null) {
+                final GlobalSunJaasKerberosConfig krb5Config = new GlobalSunJaasKerberosConfig();
+                krb5Config.setKrbConfLocation(krb5ConfigFile.getAbsolutePath());
+                krb5Config.afterPropertiesSet();
+            }
+
+            // Create ticket validator to inject into KerberosServiceAuthenticationProvider
+            SunJaasKerberosTicketValidator ticketValidator = new SunJaasKerberosTicketValidator();
+            ticketValidator.setServicePrincipal(properties.getKerberosSpnegoPrincipal());
+            ticketValidator.setKeyTabLocation(new FileSystemResource(properties.getKerberosSpnegoKeytabLocation()));
+            ticketValidator.afterPropertiesSet();
+
+            kerberosTicketValidator = ticketValidator;
+
+        }
+
+        return kerberosTicketValidator;
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosUserDetailsService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosUserDetailsService.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosUserDetailsService.java
new file mode 100644
index 0000000..5471906
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosUserDetailsService.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.web.security.authentication.kerberos;
+
+import org.springframework.security.core.authority.AuthorityUtils;
+import org.springframework.security.core.userdetails.User;
+import org.springframework.security.core.userdetails.UserDetails;
+import org.springframework.security.core.userdetails.UserDetailsService;
+import org.springframework.security.core.userdetails.UsernameNotFoundException;
+
+public class KerberosUserDetailsService implements UserDetailsService {
+
+    @Override
+    public UserDetails loadUserByUsername(String username) throws UsernameNotFoundException {
+        return new User(
+                username,
+                "notUsed",
+                true,
+                true,
+                true,
+                true,
+                AuthorityUtils.createAuthorityList("ROLE_USER"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/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 9631efc..2a1856e 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
@@ -62,6 +62,11 @@ public class X509IdentityProvider implements IdentityProvider {
                     "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.";
         }
+
+        @Override
+        public AuthType getAuthType() {
+            return AuthType.OTHER.httpAuthScheme("TLS-client-cert");
+        }
     };
 
     private X509PrincipalExtractor principalExtractor;
@@ -153,7 +158,10 @@ public class X509IdentityProvider implements IdentityProvider {
     }
 
     @Override
-    public void onConfigured(IdentityProviderConfigurationContext configurationContext) throws SecurityProviderCreationException {}
+    public void onConfigured(IdentityProviderConfigurationContext configurationContext) throws SecurityProviderCreationException {
+        throw new SecurityProviderCreationException(X509IdentityProvider.class.getSimpleName() +
+                " does not currently support being loaded via IdentityProviderFactory");
+    }
 
     @Override
     public void preDestruction() throws SecurityProviderDestructionException {}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/main/resources/META-INF/services/org.apache.nifi.registry.security.authentication.IdentityProvider
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/resources/META-INF/services/org.apache.nifi.registry.security.authentication.IdentityProvider b/nifi-registry-web-api/src/main/resources/META-INF/services/org.apache.nifi.registry.security.authentication.IdentityProvider
new file mode 100644
index 0000000..ea80a03
--- /dev/null
+++ b/nifi-registry-web-api/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.web.security.authentication.kerberos.KerberosIdentityProvider
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureKerberosIT.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureKerberosIT.java b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureKerberosIT.java
new file mode 100644
index 0000000..550c729
--- /dev/null
+++ b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureKerberosIT.java
@@ -0,0 +1,173 @@
+/*
+ * 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.api;
+
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.NiFiRegistryTestApiApplication;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.skyscreamer.jsonassert.JSONAssert;
+import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.context.annotation.Import;
+import org.springframework.context.annotation.Primary;
+import org.springframework.context.annotation.Profile;
+import org.springframework.security.authentication.BadCredentialsException;
+import org.springframework.security.kerberos.authentication.KerberosTicketValidation;
+import org.springframework.security.kerberos.authentication.KerberosTicketValidator;
+import org.springframework.test.context.jdbc.Sql;
+import org.springframework.test.context.junit4.SpringRunner;
+
+import javax.ws.rs.core.Response;
+import java.io.UnsupportedEncodingException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.Base64;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Deploy the Web API Application using an embedded Jetty Server for local integration testing, with the follow characteristics:
+ *
+ * - A NiFiRegistryProperties has to be explicitly provided to the ApplicationContext using a profile unique to this test suite.
+ * - A NiFiRegistryClientConfig has been configured to create a client capable of completing one-way TLS
+ * - The database is embed H2 using volatile (in-memory) persistence
+ * - Custom SQL is clearing the DB before each test method by default, unless method overrides this behavior
+ */
+@RunWith(SpringRunner.class)
+@SpringBootTest(
+        webEnvironment = SpringBootTest.WebEnvironment.RANDOM_PORT,
+        properties = "spring.profiles.include=ITSecureKerberos")
+@Sql(executionPhase = Sql.ExecutionPhase.BEFORE_TEST_METHOD, scripts = "classpath:db/clearDB.sql")
+public class SecureKerberosIT extends IntegrationTestBase {
+
+    private static final String validKerberosTicket = "authenticate_me";
+    private static final String invalidKerberosTicket = "do_not_authenticate_me";
+
+    public static class MockKerberosTicketValidator implements KerberosTicketValidator {
+
+        @Override
+        public KerberosTicketValidation validateTicket(byte[] token) throws BadCredentialsException {
+
+            boolean validTicket;
+            try {
+                 validTicket = Arrays.equals(validKerberosTicket.getBytes("UTF-8"), token);
+            } catch (UnsupportedEncodingException e) {
+                throw new IllegalStateException(e);
+            }
+
+            if (!validTicket) {
+                throw new BadCredentialsException(MockKerberosTicketValidator.class.getSimpleName() + " does not validate token");
+            }
+
+            return new KerberosTicketValidation(
+                    "kerberosUser@LOCALHOST",
+                    "HTTP/localhsot@LOCALHOST",
+                    null,
+                    null);
+        }
+    }
+
+    @Configuration
+    @Profile("ITSecureKerberos")
+    @Import({NiFiRegistryTestApiApplication.class, SecureITClientConfiguration.class})
+    public static class KerberosSpnegoTestConfiguration {
+
+        @Primary
+        @Bean
+        public static KerberosTicketValidator kerberosTicketValidator() {
+            return new MockKerberosTicketValidator();
+        }
+
+    }
+
+    @Test
+    public void testTokenGenerationAndAccessStatus() throws Exception {
+
+        // Note: this test intentionally does not use the token generated
+        // for nifiadmin by the @Before method
+
+        // Given: the client and server have been configured correctly for Kerberos SPNEGO authentication
+        String expectedJwtPayloadJson = "{" +
+                "\"sub\":\"kerberosUser@LOCALHOST\"," +
+                "\"preferred_username\":\"kerberosUser@LOCALHOST\"," +
+                "\"iss\":\"KerberosSpnegoIdentityProvider\"" +
+                "}";
+        String expectedAccessStatusJson = "{" +
+                "\"identity\":\"kerberosUser@LOCALHOST\"," +
+                "\"status\":\"ACTIVE\"}";
+
+        // When: the /access/token/kerberos endpoint is accessed with no credentials
+        final Response tokenResponse1 = client
+                .target(createURL("/access/token/kerberos"))
+                .request()
+                .post(null, Response.class);
+
+        // Then: the server returns 401 Unauthorized with an authenticate challenge header
+        assertEquals(401, tokenResponse1.getStatus());
+        assertNotNull(tokenResponse1.getHeaders().get("www-authenticate"));
+        assertEquals(1, tokenResponse1.getHeaders().get("www-authenticate").size());
+        assertEquals("Negotiate", tokenResponse1.getHeaders().get("www-authenticate").get(0));
+
+        // When: the /access/token/kerberos endpoint is accessed again with an invalid ticket
+        String invalidTicket = new String(java.util.Base64.getEncoder().encode(invalidKerberosTicket.getBytes(Charset.forName("UTF-8"))));
+        final Response tokenResponse2 = client
+                .target(createURL("/access/token/kerberos"))
+                .request()
+                .header("Authorization", "Negotiate " + invalidTicket)
+                .post(null, Response.class);
+
+        // Then: the server returns 401 Unauthorized
+        assertEquals(401, tokenResponse2.getStatus());
+
+        // When: the /access/token/kerberos endpoint is accessed with a valid ticket
+        String validTicket = new String(Base64.getEncoder().encode(validKerberosTicket.getBytes(Charset.forName("UTF-8"))));
+        final Response tokenResponse3 = client
+                .target(createURL("/access/token/kerberos"))
+                .request()
+                .header("Authorization", "Negotiate " + validTicket)
+                .post(null, Response.class);
+
+        // Then: the server returns 200 OK with a JWT in the body
+        assertEquals(201, tokenResponse3.getStatus());
+        String token = tokenResponse3.readEntity(String.class);
+        assertTrue(StringUtils.isNotEmpty(token));
+        String[] jwtParts = token.split("\\.");
+        assertEquals(3, jwtParts.length);
+        String jwtPayload = new String(Base64.getDecoder().decode(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);
+
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/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 bdd8e11..e84ee2b 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
@@ -25,7 +25,6 @@ import org.apache.nifi.registry.model.authorization.Tenant;
 import org.apache.nifi.registry.properties.NiFiRegistryProperties;
 import org.apache.nifi.registry.security.authorization.Authorizer;
 import org.apache.nifi.registry.security.authorization.AuthorizerFactory;
-import org.apache.tomcat.util.codec.binary.Base64;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -47,6 +46,7 @@ import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import java.nio.charset.Charset;
 import java.util.Arrays;
+import java.util.Base64;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.stream.Stream;
@@ -59,7 +59,7 @@ import static org.junit.Assert.assertTrue;
  * Deploy the Web API Application using an embedded Jetty Server for local integration testing, with the follow characteristics:
  *
  * - A NiFiRegistryProperties has to be explicitly provided to the ApplicationContext using a profile unique to this test suite.
- * - A NiFiRegistryClientConfig has been configured to create a client capable of completing two-way TLS
+ * - A NiFiRegistryClientConfig has been configured to create a client capable of completing one-way TLS
  * - The database is embed H2 using volatile (in-memory) persistence
  * - Custom SQL is clearing the DB before each test method by default, unless method overrides this behavior
  */
@@ -115,8 +115,7 @@ public class SecureLdapIT extends IntegrationTestBase {
         String expectedJwtPayloadJson = "{" +
                 "\"sub\":\"nobel\"," +
                 "\"preferred_username\":\"nobel\"," +
-                "\"iss\":\"LdapIdentityProvider\"," +
-                "\"aud\":\"LdapIdentityProvider\"" +
+                "\"iss\":\"LdapIdentityProvider\"" +
                 "}";
         String expectedAccessStatusJson = "{" +
                 "\"identity\":\"nobel\"," +
@@ -136,7 +135,7 @@ public class SecureLdapIT extends IntegrationTestBase {
         assertTrue(StringUtils.isNotEmpty(token));
         String[] jwtParts = token.split("\\.");
         assertEquals(3, jwtParts.length);
-        String jwtPayload = new String(Base64.decodeBase64(jwtParts[1]), "UTF-8");
+        String jwtPayload = new String(Base64.getDecoder().decode(jwtParts[1]), "UTF-8");
         JSONAssert.assertEquals(expectedJwtPayloadJson, jwtPayload, false);
 
         // When: the token is returned in the Authorization header
@@ -182,7 +181,7 @@ public class SecureLdapIT extends IntegrationTestBase {
         assertTrue(StringUtils.isNotEmpty(token));
         String[] jwtParts = token.split("\\.");
         assertEquals(3, jwtParts.length);
-        String jwtPayload = new String(Base64.decodeBase64(jwtParts[1]), "UTF-8");
+        String jwtPayload = new String(Base64.getDecoder().decode(jwtParts[1]), "UTF-8");
         JSONAssert.assertEquals(expectedJwtPayloadJson, jwtPayload, false);
 
         // When: the token is returned in the Authorization header
@@ -434,7 +433,7 @@ public class SecureLdapIT extends IntegrationTestBase {
 
     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"))));
+        final String base64credentials =  new String(Base64.getEncoder().encode(credentials.getBytes(Charset.forName("UTF-8"))));
         return base64credentials;
     }
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/test/resources/application-ITSecureKerberos.properties
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/application-ITSecureKerberos.properties b/nifi-registry-web-api/src/test/resources/application-ITSecureKerberos.properties
new file mode 100644
index 0000000..6ce3665
--- /dev/null
+++ b/nifi-registry-web-api/src/test/resources/application-ITSecureKerberos.properties
@@ -0,0 +1,36 @@
+#
+# 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.
+#
+
+
+# Properties for Spring Boot integration tests
+# Documentation for common Spring Boot application properties can be found at:
+# https://docs.spring.io/spring-boot/docs/current/reference/html/common-application-properties.html
+
+
+# Custom (non-standard to Spring Boot) properties
+nifi.registry.properties.file: src/test/resources/conf/secure-kerberos/nifi-registry.properties
+nifi.registry.client.properties.file: src/test/resources/conf/secure-kerberos/nifi-registry-client.properties
+
+
+# Embedded Server SSL Context Config
+#server.ssl.client-auth: need  # LDAP-configured server does not require two-way TLS
+server.ssl.key-store: ./target/test-classes/keys/localhost-ks.jks
+server.ssl.key-store-password: localhostKeystorePassword
+server.ssl.key-password: localhostKeystorePassword
+server.ssl.protocol: TLS
+server.ssl.trust-store: ./target/test-classes/keys/localhost-ts.jks
+server.ssl.trust-store-password: localhostTruststorePassword

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/authorizers.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/authorizers.xml b/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/authorizers.xml
new file mode 100644
index 0000000..6b42fa2
--- /dev/null
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/authorizers.xml
@@ -0,0 +1,102 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  ~ 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.
+  -->
+<!--
+    This file lists the userGroupProviders, accessPolicyProviders, and authorizers to use when running securely. In order
+    to use a specific authorizer it must be configured here and its identifier must be specified in the nifi.properties file.
+    If the authorizer is a managedAuthorizer, it may need to be configured with an accessPolicyProvider and an userGroupProvider.
+    This file allows for configuration of them, but they must be configured in order:
+
+    ...
+    all userGroupProviders
+    all accessPolicyProviders
+    all Authorizers
+    ...
+-->
+<authorizers>
+
+    <!--
+        The FileUserGroupProvider will provide support for managing users and groups which is backed by a file
+        on the local file system.
+
+        - Users File - The file where the FileUserGroupProvider will store users and groups.
+
+        - Initial User Identity [unique key] - The identity of a users and systems to seed the Users File. The name of
+            each property must be unique, for example: "Initial User Identity A", "Initial User Identity B",
+            "Initial User Identity C" or "Initial User Identity 1", "Initial User Identity 2", "Initial User Identity 3"
+
+            NOTE: Any identity mapping rules specified in nifi.properties will also be applied to the user identities,
+            so the values should be the unmapped identities (i.e. full DN from a certificate).
+    -->
+    <userGroupProvider>
+        <identifier>file-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.file.FileUserGroupProvider</class>
+        <property name="Users File">./target/test-classes/conf/secure-kerberos/users.xml</property>
+        <property name="Initial User Identity 1">kerberosUser@LOCALHOST</property>
+    </userGroupProvider>
+
+    <!--
+        The FileAccessPolicyProvider will provide support for managing access policies which is backed by a file
+        on the local file system.
+
+        - User Group Provider - The identifier for an User Group Provider defined above that will be used to access
+            users and groups for use in the managed access policies.
+
+        - Authorizations File - The file where the FileAccessPolicyProvider will store policies.
+
+        - Initial Admin Identity - The identity of an initial admin user that will be granted access to the UI and
+            given the ability to create additional users, groups, and policies. The value of this property could be
+            a DN when using certificates or LDAP. This property will only be used when there
+            are no other policies defined.
+
+            NOTE: Any identity mapping rules specified in nifi.properties will also be applied to the initial admin identity,
+            so the value should be the unmapped identity. This identity must be found in the configured User Group Provider.
+
+        - Node Identity [unique key] - The identity of a NiFi cluster node. When clustered, a property for each node
+            should be defined, so that every node knows about every other node. If not clustered these properties can be ignored.
+            The name of each property must be unique, for example for a three node cluster:
+            "Node Identity A", "Node Identity B", "Node Identity C" or "Node Identity 1", "Node Identity 2", "Node Identity 3"
+
+            NOTE: Any identity mapping rules specified in nifi.properties will also be applied to the node identities,
+            so the values should be the unmapped identities (i.e. full DN from a certificate). This identity must be found
+            in the configured User Group Provider.
+    -->
+    <accessPolicyProvider>
+        <identifier>file-access-policy-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.file.FileAccessPolicyProvider</class>
+        <property name="User Group Provider">file-user-group-provider</property>
+        <property name="Authorizations File">./target/test-classes/conf/secure-kerberos/authorizations.xml</property>
+        <property name="Initial Admin Identity">kerberosUser@LOCALHOST</property>
+
+        <!--<property name="Node Identity 1"></property>-->
+    </accessPolicyProvider>
+
+    <!--
+        The StandardManagedAuthorizer. This authorizer implementation must be configured with the
+        Access Policy Provider which it will use to access and manage users, groups, and policies.
+        These users, groups, and policies will be used to make all access decisions during authorization
+        requests.
+
+        - Access Policy Provider - The identifier for an Access Policy Provider defined above.
+    -->
+    <authorizer>
+        <identifier>managed-authorizer</identifier>
+        <class>org.apache.nifi.registry.security.authorization.StandardManagedAuthorizer</class>
+        <property name="Access Policy Provider">file-access-policy-provider</property>
+    </authorizer>
+
+</authorizers>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/identity-providers.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/identity-providers.xml b/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/identity-providers.xml
new file mode 100644
index 0000000..85f1957
--- /dev/null
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/identity-providers.xml
@@ -0,0 +1,31 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  ~ 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.
+  -->
+<!--
+    This file lists the login identity providers to use when running securely. In order
+    to use a specific provider it must be configured here and it's identifier
+    must be specified in the nifi.properties file.
+-->
+<identityProviders>
+
+    <!-- This test conf is for KerberosSpnegoIdentityProvider,
+         which is configured in nifi-registry.properties and loaded as an auto-scanned Spring Bean.
+
+         This is not intended for KerberosIdentityProvider,
+         which would be loaded from here using IdentityProviderFactory -->
+
+</identityProviders>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/nifi-registry-client.properties
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/nifi-registry-client.properties b/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/nifi-registry-client.properties
new file mode 100644
index 0000000..f431ccc
--- /dev/null
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/nifi-registry-client.properties
@@ -0,0 +1,22 @@
+#
+# 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.
+#
+
+# client security properties #
+# Don't use a client cert for one-way TLS. Client identity will be provided via Kerberos SPNEGO to get JWT
+nifi.registry.security.truststore=./target/test-classes/keys/localhost-ts.jks
+nifi.registry.security.truststoreType=JKS
+nifi.registry.security.truststorePasswd=localhostTruststorePassword

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/nifi-registry.properties
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/nifi-registry.properties b/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/nifi-registry.properties
new file mode 100644
index 0000000..3d5c122
--- /dev/null
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-kerberos/nifi-registry.properties
@@ -0,0 +1,36 @@
+#
+# 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.
+#
+
+# web properties #
+nifi.registry.web.https.host=localhost
+nifi.registry.web.https.port=0
+
+# security properties #
+#
+# ** Server KeyStore and TrustStore configuration set in Spring profile properties for embedded Jetty **
+#
+nifi.registry.security.authorizers.configuration.file=./target/test-classes/conf/secure-kerberos/authorizers.xml
+nifi.registry.security.authorizer=managed-authorizer
+
+# providers properties #
+nifi.registry.providers.configuration.file=./target/test-classes/conf/providers.xml
+
+# kerberos properties # (aside from expiration, these don't actually matter as the KerberosServiceAuthenticationProvider will be mocked)
+nifi.registry.kerberos.krb5.file=/path/to/krb5.conf
+nifi.registry.kerberos.spnego.authentication.expiration=12 hours
+nifi.registry.kerberos.spnego.principal=HTTP/localhost@LOCALHOST
+nifi.registry.kerberos.spnego.keytab.location=/path/to/keytab

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/test/resources/conf/secure-ldap/nifi-registry-client.properties
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/conf/secure-ldap/nifi-registry-client.properties b/nifi-registry-web-api/src/test/resources/conf/secure-ldap/nifi-registry-client.properties
index 929e1a7..68cb0f9 100644
--- a/nifi-registry-web-api/src/test/resources/conf/secure-ldap/nifi-registry-client.properties
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-ldap/nifi-registry-client.properties
@@ -16,10 +16,7 @@
 #
 
 # client security properties #
-#nifi.registry.security.keystore=./target/test-classes/keys/client-ks.jks
-#nifi.registry.security.keystoreType=JKS
-#nifi.registry.security.keystorePasswd=clientKeystorePassword
-#nifi.registry.security.keyPasswd=u1Pass
+# Don't use a client cert for one-way TLS. Client identity will be provided via LDAP user/pass to get JWT
 nifi.registry.security.truststore=./target/test-classes/keys/localhost-ts.jks
 nifi.registry.security.truststoreType=JKS
 nifi.registry.security.truststorePasswd=localhostTruststorePassword


[2/2] nifi-registry git commit: NIFIREG-52 Add Kerberos Support

Posted by bb...@apache.org.
NIFIREG-52 Add Kerberos Support

- KerberosIdentityProvider: an IdentityProvider extension for username/password login
- KerberosSpnegoIdentityProvider: authenticates Kerberos tickets over SPNEGO exchange with client and mutual Kerberos server
- Adds NiFiRegistryProperties fields for configuring KerberosSpnegoIdentityProvider
- Adds a dedicated endpoint for clients to use to generate a JWT using a Kerberos Ticket (backed by KerberosSpnegoIP)
- Adds a new endpoint (POST /acces/token) that attempts to do server-side auto detection of client authentication method
- Adds NotAllowedExceptionMapper that returns 405 for NotAllowedExceptions thrown in the Jersey framework
- Adds UnathorizedException and UnathorizedExceptionMapper for returning 401 responses with WWW-Authenticate challenges
- Refines IdentityProvider interface by connecting WWW-Authenticate challenges to IdentityProviderUsage object

This closes #41.

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/58925377
Tree: http://git-wip-us.apache.org/repos/asf/nifi-registry/tree/58925377
Diff: http://git-wip-us.apache.org/repos/asf/nifi-registry/diff/58925377

Branch: refs/heads/master
Commit: 5892537789b617a6ec6380eb79c0c979fe9e1933
Parents: ef8ba12
Author: Kevin Doran <kd...@gmail.com>
Authored: Thu Nov 16 10:30:42 2017 -0500
Committer: Bryan Bende <bb...@apache.org>
Committed: Tue Nov 21 12:50:39 2017 -0500

----------------------------------------------------------------------
 nifi-registry-assembly/pom.xml                  |   7 +
 .../security/ldap/LdapIdentityProvider.java     |   1 +
 .../properties/NiFiRegistryProperties.java      | 110 +++++---
 .../main/resources/conf/identity-providers.xml  |  16 ++
 .../resources/conf/nifi-registry.properties     |  21 +-
 .../BasicAuthIdentityProvider.java              |  17 +-
 .../BearerAuthIdentityProvider.java             |   5 +
 .../authentication/IdentityProviderUsage.java   | 103 ++++++++
 .../registry/security/util/CryptoUtils.java     |  75 ++++++
 nifi-registry-web-api/pom.xml                   |  15 ++
 .../web/NiFiRegistryResourceConfig.java         |   7 +
 .../nifi/registry/web/api/AccessResource.java   | 256 ++++++++++++++++---
 .../web/exception/UnauthorizedException.java    |  74 ++++++
 .../web/mapper/NotAllowedExceptionMapper.java   |  46 ++++
 .../web/mapper/UnauthorizedExceptionMapper.java |  56 ++++
 .../security/NiFiRegistrySecurityConfig.java    |  14 +-
 .../IdentityAuthenticationFilter.java           |  67 +----
 .../authentication/jwt/JwtIdentityProvider.java |  19 +-
 .../security/authentication/jwt/JwtService.java |  52 ++--
 .../kerberos/KerberosIdentityProvider.java      | 111 ++++++++
 .../kerberos/KerberosSpnegoFactory.java         |  67 +++++
 .../KerberosSpnegoIdentityProvider.java         | 177 +++++++++++++
 .../KerberosTicketValidatorFactory.java         |  69 +++++
 .../kerberos/KerberosUserDetailsService.java    |  38 +++
 .../x509/X509IdentityProvider.java              |  10 +-
 ...try.security.authentication.IdentityProvider |  15 ++
 .../nifi/registry/web/api/SecureKerberosIT.java | 173 +++++++++++++
 .../nifi/registry/web/api/SecureLdapIT.java     |  13 +-
 .../application-ITSecureKerberos.properties     |  36 +++
 .../conf/secure-kerberos/authorizers.xml        | 102 ++++++++
 .../conf/secure-kerberos/identity-providers.xml |  31 +++
 .../nifi-registry-client.properties             |  22 ++
 .../secure-kerberos/nifi-registry.properties    |  36 +++
 .../secure-ldap/nifi-registry-client.properties |   5 +-
 34 files changed, 1674 insertions(+), 192 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-assembly/pom.xml b/nifi-registry-assembly/pom.xml
index 41f66b8..45df1c4 100644
--- a/nifi-registry-assembly/pom.xml
+++ b/nifi-registry-assembly/pom.xml
@@ -163,6 +163,13 @@
         <!-- nifi-registry.properties: database properties -->
         <nifi.registry.db.directory>./database</nifi.registry.db.directory>
         <nifi.registry.db.url.append>;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE</nifi.registry.db.url.append>
+
+        <!-- nifi-registry.properties: kerberos properties -->
+        <nifi.registry.kerberos.krb5.file />
+        <nifi.registry.kerberos.spnego.principal />
+        <nifi.registry.kerberos.spnego.keytab.location />
+        <nifi.registry.kerberos.spnego.authentication.expiration>12 hours</nifi.registry.kerberos.spnego.authentication.expiration>
+
     </properties>
 
     <profiles>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/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 bebe2d5..4427ed9 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
@@ -248,6 +248,7 @@ public class LdapIdentityProvider extends BasicAuthIdentityProvider implements I
             // perform the authentication
             final UsernamePasswordAuthenticationToken token = new UsernamePasswordAuthenticationToken(username, credentials);
             final Authentication authentication = ldapAuthenticationProvider.authenticate(token);
+            logger.debug("Created authentication token: {}", token.toString());
 
             // use dn if configured
             if (IdentityStrategy.USE_DN.equals(identityStrategy)) {

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryProperties.java
----------------------------------------------------------------------
diff --git a/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryProperties.java b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryProperties.java
index b0090c0..bf18609 100644
--- a/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryProperties.java
+++ b/nifi-registry-properties/src/main/java/org/apache/nifi/registry/properties/NiFiRegistryProperties.java
@@ -61,12 +61,19 @@ public class NiFiRegistryProperties extends Properties {
     public static final String DATABASE_DIRECTORY = "nifi.registry.db.directory";
     public static final String DATABASE_URL_APPEND = "nifi.registry.db.url.append";
 
+    // Kerberos properties
+    public static final String KERBEROS_KRB5_FILE = "nifi.registry.kerberos.krb5.file";
+    public static final String KERBEROS_SPNEGO_PRINCIPAL = "nifi.registry.kerberos.spnego.principal";
+    public static final String KERBEROS_SPNEGO_KEYTAB_LOCATION = "nifi.registry.kerberos.spnego.keytab.location";
+    public static final String KERBEROS_SPNEGO_AUTHENTICATION_EXPIRATION = "nifi.registry.kerberos.spnego.authentication.expiration";
+
     // Defaults
     public static final String DEFAULT_WEB_WORKING_DIR = "./work/jetty";
     public static final String DEFAULT_WAR_DIR = "./lib";
     public static final String DEFAULT_PROVIDERS_CONFIGURATION_FILE = "./conf/providers.xml";
     public static final String DEFAULT_SECURITY_AUTHORIZERS_CONFIGURATION_FILE = "./conf/authorizers.xml";
     public static final String DEFAULT_SECURITY_IDENTITY_PROVIDER_CONFIGURATION_FILE = "./conf/identity-providers.xml";
+    public static final String DEFAULT_AUTHENTICATION_EXPIRATION = "12 hours";
 
     public int getWebThreads() {
         int webThreads = 200;
@@ -79,15 +86,7 @@ public class NiFiRegistryProperties extends Properties {
     }
 
     public Integer getPort() {
-        final String rawPort = getProperty(WEB_HTTP_PORT);
-        if (StringUtils.isBlank(rawPort)) {
-            return null;
-        }
-        try {
-            return Integer.parseInt(rawPort);
-        } catch (final NumberFormatException nfe) {
-            throw new IllegalStateException(String.format("%s must be an integer value.", WEB_HTTP_PORT));
-        }
+        return getPropertyAsInteger(WEB_HTTP_PORT);
     }
 
     public String getHttpHost() {
@@ -95,15 +94,7 @@ public class NiFiRegistryProperties extends Properties {
     }
 
     public Integer getSslPort() {
-        final String rawPort = getProperty(WEB_HTTPS_PORT);
-        if (StringUtils.isBlank(rawPort)) {
-            return null;
-        }
-        try {
-            return Integer.parseInt(rawPort);
-        } catch (final NumberFormatException nfe) {
-            throw new IllegalStateException(String.format("%s must be an integer value.", WEB_HTTPS_PORT));
-        }
+        return getPropertyAsInteger(WEB_HTTPS_PORT);
     }
 
     public String getHttpsHost() {
@@ -156,12 +147,7 @@ public class NiFiRegistryProperties extends Properties {
     }
 
     public File getProvidersConfigurationFile() {
-        final String value = getProperty(PROVIDERS_CONFIGURATION_FILE);
-        if (StringUtils.isBlank(value)) {
-            return new File(DEFAULT_PROVIDERS_CONFIGURATION_FILE);
-        } else {
-            return new File(value);
-        }
+        return getPropertyAsFile(PROVIDERS_CONFIGURATION_FILE, DEFAULT_PROVIDERS_CONFIGURATION_FILE);
     }
 
     public String getDatabaseDirectory() {
@@ -173,21 +159,31 @@ public class NiFiRegistryProperties extends Properties {
     }
 
     public File getAuthorizersConfigurationFile() {
-        final String value = getProperty(SECURITY_AUTHORIZERS_CONFIGURATION_FILE);
-        if (StringUtils.isBlank(value)) {
-            return new File(DEFAULT_SECURITY_AUTHORIZERS_CONFIGURATION_FILE);
-        } else {
-            return new File(value);
-        }
+        return getPropertyAsFile(SECURITY_AUTHORIZERS_CONFIGURATION_FILE, DEFAULT_SECURITY_AUTHORIZERS_CONFIGURATION_FILE);
     }
 
     public File getIdentityProviderConfigurationFile() {
-        final String value = getProperty(SECURITY_IDENTITY_PROVIDERS_CONFIGURATION_FILE);
-        if (StringUtils.isBlank(value)) {
-            return new File(DEFAULT_SECURITY_IDENTITY_PROVIDER_CONFIGURATION_FILE);
-        } else {
-            return new File(value);
-        }
+        return getPropertyAsFile(SECURITY_IDENTITY_PROVIDERS_CONFIGURATION_FILE, DEFAULT_SECURITY_IDENTITY_PROVIDER_CONFIGURATION_FILE);
+    }
+
+    public File getKerberosConfigurationFile() {
+        return getPropertyAsFile(KERBEROS_KRB5_FILE);
+    }
+
+    public String getKerberosSpnegoAuthenticationExpiration() {
+        return getProperty(KERBEROS_SPNEGO_AUTHENTICATION_EXPIRATION, DEFAULT_AUTHENTICATION_EXPIRATION);
+    }
+
+    public String getKerberosSpnegoPrincipal() {
+        return getPropertyAsTrimmedString(KERBEROS_SPNEGO_PRINCIPAL);
+    }
+
+    public String getKerberosSpnegoKeytabLocation() {
+        return getPropertyAsTrimmedString(KERBEROS_SPNEGO_KEYTAB_LOCATION);
+    }
+
+    public boolean isKerberosSpnegoSupportEnabled() {
+        return !StringUtils.isBlank(getKerberosSpnegoPrincipal()) && !StringUtils.isBlank(getKerberosSpnegoKeytabLocation());
     }
 
     public Set<String> getExtensionsDirs() {
@@ -210,4 +206,46 @@ public class NiFiRegistryProperties extends Properties {
 
         return propertyNames;
     }
+
+    // Helper functions for common ways of interpreting property values
+
+    private String getPropertyAsTrimmedString(String key) {
+        final String value = getProperty(key);
+        if (!StringUtils.isBlank(value)) {
+            return value.trim();
+        } else {
+            return null;
+        }
+    }
+
+    private Integer getPropertyAsInteger(String key) {
+        final String value = getProperty(key);
+        if (StringUtils.isBlank(value)) {
+            return null;
+        }
+        try {
+            return Integer.parseInt(value);
+        } catch (final NumberFormatException nfe) {
+            throw new IllegalStateException(String.format("%s must be an integer value.", key));
+        }
+    }
+
+    private File getPropertyAsFile(String key) {
+        final String filePath = getProperty(key);
+        if (filePath != null && filePath.trim().length() > 0) {
+            return new File(filePath.trim());
+        } else {
+            return null;
+        }
+    }
+
+    private File getPropertyAsFile(String propertyKey, String defaultFileLocation) {
+        final String value = getProperty(propertyKey);
+        if (StringUtils.isBlank(value)) {
+            return new File(defaultFileLocation);
+        } else {
+            return new File(value);
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-resources/src/main/resources/conf/identity-providers.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-resources/src/main/resources/conf/identity-providers.xml b/nifi-registry-resources/src/main/resources/conf/identity-providers.xml
index 6fcaf7c..1e8cf64 100644
--- a/nifi-registry-resources/src/main/resources/conf/identity-providers.xml
+++ b/nifi-registry-resources/src/main/resources/conf/identity-providers.xml
@@ -87,4 +87,20 @@
     </provider>
     To enable the ldap-identity-provider remove 2 lines. This is 2 of 2. -->
 
+    <!--
+        Identity Provider for users logging in with username/password against a Kerberos KDC server.
+
+        'Default Realm' - Default realm to provide when user enters incomplete user principal (i.e. NIFI.APACHE.ORG).
+        'Authentication Expiration' - The duration of how long the user authentication is valid for. If the user never logs out, they will be required to log back in following this duration.
+    -->
+    <!-- To enable the kerberos-identity-provider remove 2 lines. This is 1 of 2.
+    <provider>
+        <identifier>kerberos-identity-provider</identifier>
+        <class>org.apache.nifi.registry.web.security.authentication.kerberos.KerberosIdentityProvider</class>
+        <property name="Default Realm">NIFI.APACHE.ORG</property>
+        <property name="Authentication Expiration">12 hours</property>
+        <property name="Enable Debug">false</property>
+    </provider>
+    To enable the kerberos-provider remove 2 lines. This is 2 of 2. -->
+
 </identityProviders>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
----------------------------------------------------------------------
diff --git a/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties b/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
index 7f0c7ae..28577dc 100644
--- a/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
+++ b/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
@@ -44,9 +44,24 @@ nifi.registry.db.directory=${nifi.registry.db.directory}
 nifi.registry.db.url.append=${nifi.registry.db.url.append}
 
 # extension directories #
-
 # Each property beginning with "nifi.registry.extension.dir." will be treated as location for an extension,
 # and a class loader will be created for each location, with the system class loader as the parent
-
+#
 #nifi.registry.extension.dir.1=/path/to/extension1
-#nifi.registry.extension.dir.2=/path/to/extension2
\ No newline at end of file
+#nifi.registry.extension.dir.2=/path/to/extension2
+
+# Identity Mapping Properties #
+# These properties allow normalizing user identities such that identities coming from different identity providers
+# (certificates, LDAP, Kerberos) can be treated the same internally in NiFi. The following example demonstrates normalizing
+# DNs from certificates and principals from Kerberos into a common identity string:
+#
+# nifi.registry.security.identity.mapping.pattern.dn=^CN=(.*?), OU=(.*?), O=(.*?), L=(.*?), ST=(.*?), C=(.*?)$
+# nifi.registry.security.identity.mapping.value.dn=$1@$2
+# nifi.registry.security.identity.mapping.pattern.kerb=^(.*?)/instance@(.*?)$
+# nifi.registry.security.identity.mapping.value.kerb=$1@$2
+
+# kerberos properties #
+nifi.registry.kerberos.krb5.file=${nifi.registry.kerberos.krb5.file}
+nifi.registry.kerberos.spnego.principal=${nifi.registry.kerberos.spnego.principal}
+nifi.registry.kerberos.spnego.keytab.location=${nifi.registry.kerberos.spnego.keytab.location}
+nifi.registry.kerberos.spnego.authentication.expiration=${nifi.registry.kerberos.spnego.authentication.expiration}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/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
index 10a1065..64c8b8e 100644
--- 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
@@ -37,6 +37,11 @@ public abstract class BasicAuthIdentityProvider implements IdentityProvider {
                     "That is: 'Authorization: Basic <credentials>', " +
                     "where <credentials> is the base64 encoded value of '<username>:<password>'.";
         }
+
+        @Override
+        public AuthType getAuthType() {
+            return AuthType.BASIC;
+        }
     };
 
     @Override
@@ -57,16 +62,16 @@ public abstract class BasicAuthIdentityProvider implements IdentityProvider {
             return null;
         }
 
+        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;
+        }
+
         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"));

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/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
index 5d08e9a..0647782 100644
--- 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
@@ -35,6 +35,11 @@ public abstract class BearerAuthIdentityProvider implements IdentityProvider {
                     "That is: 'Authorization: Bearer <token>', " +
                     "where <token> is a value that will be validated by this identity provider.";
         }
+
+        @Override
+        public AuthType getAuthType() {
+            return AuthType.BEARER;
+        }
     };
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/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
index 98eea95..aefc97d 100644
--- 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
@@ -29,4 +29,107 @@ public interface IdentityProviderUsage {
      */
     String getText();
 
+    /**
+     * If the identity provider follows an HTTP standard auth
+     * scheme, this provides which scheme is being used
+     * (or "Other" if the identity provider follows its own scheme).
+     *
+     * In the case the scheme is well understood, such as HTTP
+     * "Basic" Auth, this may be sufficient. In other cases,
+     * {@link #getText()} should provider detailed human-readable
+     * instructions about how a client should interact with
+     * the {@link IdentityProvider}.
+     *
+     * @return an enum for the auth
+     */
+    AuthType getAuthType();
+
+    /**
+     * Standard auth types as maintained by IANA:
+     * https://www.iana.org/assignments/http-authschemes/http-authschemes.xhtml
+     *
+     * Note, draft and experimental standards are not included, nor are app-specific custom schemes.
+     * To create an enum for such a scheme, use OTHER with a custom httpAuthScheme string, e.g.:
+     *
+     * <code>AuthType myAuthType = AuthType.OTHER.httpAuthScheme("my-auth-scheme");</code>
+     */
+    enum AuthType {
+
+        /**
+         * Indicates the AuthType is unknown. Can be used in places where an AuthType is required but unknown by default.
+         */
+        UNKNOWN(0, "Unknown"),
+
+        /**
+         * HTTP Basic Auth as defined by RFC7617.
+         */
+        BASIC(1, "Basic"),
+
+        /**
+         * HTTP Bearer Auth as defined by RFC6750.
+         */
+        BEARER(2, "Bearer"),
+
+        /**
+         * HTTP Digest Auth as defined by RFC7616.
+         */
+        DIGEST(3, "Digest"),
+
+        /**
+         * HTTP Negotiate (SPNEGO) Auth as defined by RFC4559.
+         */
+        NEGOTIATE(4, "Negotiate"),
+
+        /**
+         * HTTP OAuth as defined by RFC5849
+         */
+        OAUTH(5, "OAuth"),
+
+        /**
+         * A distinct AuthType for which there is not yet a defined enumeration value.
+         * If a HTTP Auth Scheme should be set (e.g., for use in a WWW-Authenticate challenge list)
+         * use the setter, i.e.:
+         * <code>AuthType myAuthType = AuthType.OTHER.httpAuthScheme("my-auth-scheme");</code>
+         */
+        OTHER(99, "Other"),
+        ;
+
+        private final int code;
+        private String httpAuthScheme;
+
+        private AuthType(int statusCode, String httpAuthScheme) {
+            this.code = statusCode;
+            this.httpAuthScheme = httpAuthScheme;
+        }
+
+        public int getStatusCode() {
+            return this.code;
+        }
+
+        public String getHttpAuthScheme() {
+            return this.toString();
+        }
+
+        public AuthType httpAuthScheme(String httpAuthScheme) {
+            if (httpAuthScheme != null) {
+                this.httpAuthScheme = httpAuthScheme;
+            }
+            return this;
+        }
+
+        public String toString() {
+            return this.httpAuthScheme;
+        }
+
+        public static AuthType fromCode(int code) {
+            AuthType[] enumTypes = values();
+            for (AuthType s : enumTypes) {
+                if (s.code == code) {
+                    return s;
+                }
+            }
+            return null;
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/CryptoUtils.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/CryptoUtils.java b/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/CryptoUtils.java
new file mode 100644
index 0000000..cd2d3e3
--- /dev/null
+++ b/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/CryptoUtils.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.security.util;
+
+import javax.crypto.Cipher;
+import java.security.NoSuchAlgorithmException;
+
+public class CryptoUtils {
+
+    /**
+     *  Required Cipher transformations according to Java SE 8 {@link Cipher} docs
+     */
+    private static final String[] standardCryptoTransformations = {
+        "AES/CBC/NoPadding",
+        "AES/CBC/PKCS5Padding",
+        "AES/ECB/NoPadding",
+        "AES/ECB/PKCS5Padding",
+        "DES/CBC/NoPadding",
+        "DES/CBC/PKCS5Padding",
+        "DES/ECB/NoPadding",
+        "DES/ECB/PKCS5Padding",
+        "DESede/CBC/NoPadding",
+        "DESede/CBC/PKCS5Padding",
+        "DESede/ECB/NoPadding",
+        "DESede/ECB/PKCS5Padding",
+        "RSA/ECB/PKCS1Padding",
+        "RSA/ECB/OAEPWithSHA-1AndMGF1Padding",
+        "RSA/ECB/OAEPWithSHA-256AndMGF1Padding"
+    };
+
+    /**
+     * Check if cryptographic strength available in this Java Runtime is restricted.
+     *
+     * Not every Java Platform supports "unlimited strength encryption",
+     * so this convenience method provides a way to check if strength of crypto
+     * functions (i.e., max key length) is unlimited or restricted in the
+     * current Java runtime environment.
+     *
+     * @return true if it can be determined that max key lengths are less than unlimited
+     *         false if key lengths are restricted
+     *         null if max key length cannot be determined for any known Cipher transformations */
+    public static Boolean isCryptoRestricted() {
+
+        Boolean isCryptoRestricted = null;
+
+        for (String transformation : standardCryptoTransformations) {
+            try {
+                return Cipher.getMaxAllowedKeyLength(transformation) < Integer.MAX_VALUE;
+            } catch (final NoSuchAlgorithmException e) {
+                // Unexpected as we are pulling from a list of transforms that every
+                // java platform is required to support, but try the next one
+            }
+        }
+
+        // Tried every standard Cipher transformation and none were available,
+        // so crypto strength restrictions cannot be determined.
+        return null;
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/pom.xml b/nifi-registry-web-api/pom.xml
index 203b58d..8b53496 100644
--- a/nifi-registry-web-api/pom.xml
+++ b/nifi-registry-web-api/pom.xml
@@ -113,6 +113,21 @@
             <artifactId>spring-boot-starter-actuator</artifactId>
             <version>${spring.boot.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.springframework.security.kerberos</groupId>
+            <artifactId>spring-security-kerberos-core</artifactId>
+            <version>1.0.1.RELEASE</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.springframework</groupId>
+                    <artifactId>spring-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.springframework.security</groupId>
+                    <artifactId>spring-security-core</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
         <!-- Must be marked provided in order to produce a correct WAR -->
         <dependency>
             <groupId>org.springframework.boot</groupId>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryResourceConfig.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryResourceConfig.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryResourceConfig.java
index 503c948..2fe9fcd 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryResourceConfig.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistryResourceConfig.java
@@ -66,7 +66,14 @@ public class NiFiRegistryResourceConfig extends ResourceConfig {
         register(ResourceResource.class);
         register(TenantResource.class);
 
+        // include bean validation errors in response
         property(ServerProperties.BV_SEND_ERROR_IN_RESPONSE, true);
+
+        // this is necessary for the /access/token/kerberos endpoint to work correctly
+        // when sending 401 Unauthorized with a WWW-Authenticate: Negotiate header.
+        // if this value needs to be changed, kerberos authentication needs to move to filter chain
+        // so it can directly set the HttpServletResponse instead of indirectly through a JAX-RS Response
+        property(ServerProperties.RESPONSE_SET_STATUS_OVER_SEND_ERROR, true);
     }
 
     // Disable default SpringMVC filter beans that are not compatible with Jersey

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/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 84693b1..5cd1ccd 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
@@ -16,6 +16,7 @@
  */
 package org.apache.nifi.registry.web.api;
 
+import io.jsonwebtoken.JwtException;
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
@@ -23,16 +24,20 @@ 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.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.IdentityProvider;
+import org.apache.nifi.registry.security.authentication.IdentityProviderUsage;
 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.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.exception.UnauthorizedException;
 import org.apache.nifi.registry.web.security.authentication.jwt.JwtService;
+import org.apache.nifi.registry.web.security.authentication.kerberos.KerberosSpnegoIdentityProvider;
+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;
@@ -49,6 +54,10 @@ 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.ArrayList;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
 
 @Component
 @Path("/access")
@@ -60,15 +69,23 @@ public class AccessResource extends ApplicationResource {
 
     private static final Logger logger = LoggerFactory.getLogger(AccessResource.class);
 
-    private IdentityProvider identityProvider;
+    private NiFiRegistryProperties properties;
     private JwtService jwtService;
+    private X509IdentityProvider x509IdentityProvider;
+    private KerberosSpnegoIdentityProvider kerberosSpnegoIdentityProvider;
+    private IdentityProvider identityProvider;
 
     @Autowired
     public AccessResource(
-            AuthorizationService authorizationService,
+            NiFiRegistryProperties properties,
             JwtService jwtService,
+            X509IdentityProvider x509IdentityProvider,
+            KerberosSpnegoIdentityProvider kerberosSpnegoIdentityProvider,
             IdentityProvider identityProvider) {
+        this.properties = properties;
         this.jwtService = jwtService;
+        this.x509IdentityProvider = x509IdentityProvider;
+        this.kerberosSpnegoIdentityProvider = kerberosSpnegoIdentityProvider;
         this.identityProvider = identityProvider;
     }
 
@@ -86,9 +103,6 @@ public class AccessResource extends ApplicationResource {
             response = AccessStatus.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 might be running unsecured.") })
     public Response getAccessStatus(@Context HttpServletRequest httpServletRequest) {
         // only consider user specific access over https
@@ -112,6 +126,7 @@ public class AccessResource extends ApplicationResource {
         return generateOkResponse(accessStatus).build();
     }
 
+
     /**
      * Creates a token for accessing the REST API.
      *
@@ -121,6 +136,71 @@ public class AccessResource extends ApplicationResource {
     @POST
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.TEXT_PLAIN)
+    @Path("/token")
+    @ApiOperation(
+            value = "Creates a token for accessing the REST API via auto-detected method of verifying client identity claim credentials",
+            notes = "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 = 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 createAccessTokenByTryingAllProviders(@Context HttpServletRequest httpServletRequest) {
+
+        // only support access tokens when communicating over HTTPS
+        if (!httpServletRequest.isSecure()) {
+            throw new IllegalStateException("Access tokens are only issued over HTTPS");
+        }
+
+        List<IdentityProvider> identityProviderWaterfall = generateIdentityProviderWaterfall();
+
+        String token = null;
+        for (IdentityProvider provider : identityProviderWaterfall) {
+
+            AuthenticationRequest authenticationRequest = identityProvider.extractCredentials(httpServletRequest);
+            if (authenticationRequest == null) {
+                continue;
+            }
+            try {
+                token = createAccessToken(identityProvider, authenticationRequest);
+                break;
+            } catch (final InvalidCredentialsException ice){
+                logger.debug("{}: the supplied client credentials are invalid.", identityProvider.getClass().getSimpleName());
+                logger.debug("", ice);
+            }
+
+        }
+
+        if (StringUtils.isEmpty(token)) {
+            List<IdentityProviderUsage.AuthType> acceptableAuthTypes = identityProviderWaterfall.stream()
+                    .map(IdentityProvider::getUsageInstructions)
+                    .map(IdentityProviderUsage::getAuthType)
+                    .filter(Objects::nonNull)
+                    .distinct()
+                    .collect(Collectors.toList());
+
+            throw new UnauthorizedException("Client credentials are missing or invalid according to all configured identity providers.")
+                    .withAuthenticateChallenge(acceptableAuthTypes);
+        }
+
+        // build the response
+        final URI uri = URI.create(generateResourceUri("access", "token"));
+        return generateCreatedResponse(uri, token).build();
+    }
+
+    /**
+     * Creates a token for accessing the REST API.
+     *
+     * @param httpServletRequest the servlet request
+     * @return A JWT (string)
+     */
+    @POST
+    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
+    @Produces(MediaType.TEXT_PLAIN)
     @Path("/token/login")
     @ApiOperation(
             value = "Creates a token for accessing the REST API via username/password",
@@ -132,7 +212,6 @@ public class AccessResource extends ApplicationResource {
     @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 username/password."),
             @ApiResponse(code = 500, message = HttpStatusMessages.MESSAGE_500) })
     public Response createAccessTokenUsingFormLogin(
@@ -155,24 +234,68 @@ public class AccessResource extends ApplicationResource {
             throw new IllegalArgumentException("The username and password must be specified");
         }
 
-        final AuthenticationResponse authenticationResponse;
-
+        // generate JWT for response
+        AuthenticationRequest authenticationRequest = new UsernamePasswordAuthenticationRequest(username, password);
+        final String token;
         try {
-            // attempt to authenticate
-            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);
+             token = createAccessToken(identityProvider, authenticationRequest);
+        } catch (final InvalidCredentialsException ice){
+            throw new UnauthorizedException("The supplied client credentials are not valid.", ice)
+                    .withAuthenticateChallenge("form-login");
         }
 
-        // generate JWT for response
-        final String token = jwtService.generateSignedToken(authenticationResponse);
+        // form the response
+        final URI uri = URI.create(generateResourceUri("access", "token"));
+        return generateCreatedResponse(uri, token).build();
+    }
+
+    @POST
+    @Consumes(MediaType.WILDCARD)
+    @Produces(MediaType.TEXT_PLAIN)
+    @Path("/token/kerberos")
+    @ApiOperation(
+            value = "Creates a token for accessing the REST API via Kerberos Service Tickets or SPNEGO Tokens (which includes Kerberos Service Tickets)",
+            notes = "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 = 409, message = HttpStatusMessages.MESSAGE_409 + " The NiFi Registry may not be configured to support login Kerberos credentials."),
+            @ApiResponse(code = 500, message = HttpStatusMessages.MESSAGE_500) })
+    public Response createAccessTokenUsingKerberosTicket(@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 (!properties.isKerberosSpnegoSupportEnabled() || kerberosSpnegoIdentityProvider == null) {
+            throw new IllegalStateException("Kerberos service ticket login not supported by this NiFi Registry");
+        }
+
+        AuthenticationRequest authenticationRequest = kerberosSpnegoIdentityProvider.extractCredentials(httpServletRequest);
+
+        if (authenticationRequest == null) {
+            throw new UnauthorizedException("The client credentials are missing from the request.")
+                    .withAuthenticateChallenge(kerberosSpnegoIdentityProvider.getUsageInstructions().getAuthType());
+        }
+
+        final String token;
+        try {
+            token = createAccessToken(kerberosSpnegoIdentityProvider, authenticationRequest);
+        } catch (final InvalidCredentialsException ice){
+            throw new UnauthorizedException("The supplied client credentials are not valid.", ice)
+                    .withAuthenticateChallenge(kerberosSpnegoIdentityProvider.getUsageInstructions().getAuthType());
+        }
 
         // build the response
         final URI uri = URI.create(generateResourceUri("access", "token"));
         return generateCreatedResponse(uri, token).build();
+
     }
 
     /**
@@ -188,7 +311,7 @@ public class AccessResource extends ApplicationResource {
     @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 exact format of the user credentials expected by the custom identity provider can be discovered by 'GET /access/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>'.",
@@ -197,7 +320,6 @@ public class AccessResource extends ApplicationResource {
     @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) {
@@ -212,24 +334,25 @@ public class AccessResource extends ApplicationResource {
             throw new IllegalStateException("Custom login not supported by this NiFi Registry");
         }
 
-        final AuthenticationResponse authenticationResponse;
+        AuthenticationRequest authenticationRequest = identityProvider.extractCredentials(httpServletRequest);
 
-        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);
+        if (authenticationRequest == null) {
+            throw new UnauthorizedException("The client credentials are missing from the request.")
+                    .withAuthenticateChallenge(identityProvider.getUsageInstructions().getAuthType());
         }
 
-        // generate JWT for response
-        final String token = jwtService.generateSignedToken(authenticationResponse);
+        final String token;
+        try {
+            token = createAccessToken(identityProvider, authenticationRequest);
+        } catch (InvalidCredentialsException ice) {
+            throw new UnauthorizedException("The supplied client credentials are not valid.", ice)
+                    .withAuthenticateChallenge(identityProvider.getUsageInstructions().getAuthType());
+        }
 
         // build the response
         final URI uri = URI.create(generateResourceUri("access", "token"));
         return generateCreatedResponse(uri, token).build();
+
     }
 
     /**
@@ -243,7 +366,7 @@ public class AccessResource extends ApplicationResource {
     @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",
+            value = "Provides a description of how the currently configured identity provider expects credentials to be passed to POST /access/token/identity-provider",
             response = String.class
     )
     @ApiResponses({
@@ -266,7 +389,7 @@ public class AccessResource extends ApplicationResource {
             return generateOkResponse(usageInstructions).build();
 
         } catch (Exception e) {
-            // If, for any reason, this identity provider does not support getUsageInstructions(), e.g., throws NotImplemented Exception.
+            // If, for any reason, this identity provider does not support getUsageInstructions(), e.g., returns null or throws NotImplementedException.
             return Response.status(Response.Status.NOT_IMPLEMENTED)
                     .entity("The currently configured identity provider, '" + identityProvider.getClass().getName() + "' does not provide usage instructions.")
                     .build();
@@ -286,7 +409,7 @@ public class AccessResource extends ApplicationResource {
     @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'.",
+            notes = "The user credentials should be passed in a format understood by the custom identity provider as defined by 'GET /access/token/identity-provider/usage'.",
             response = String.class
     )
     @ApiResponses({
@@ -311,16 +434,67 @@ public class AccessResource extends ApplicationResource {
 
         // 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();
+
+        if (authenticationRequest == null) {
+            throw new UnauthorizedException("The format of the credentials were not recognized by the currently configured identity provider " +
+                    "'" + identityProviderName + "'. " + identityProvider.getUsageInstructions().getText())
+                    .withAuthenticateChallenge(identityProvider.getUsageInstructions().getAuthType());
+        }
+
+
+        final String successMessage = identityProviderName + " recognized the format of the credentials in the HTTP request.";
+        return generateOkResponse(successMessage).build();
+
+    }
+
+    private String createAccessToken(IdentityProvider identityProvider, AuthenticationRequest authenticationRequest)
+            throws InvalidCredentialsException, AdministrationException {
+
+        final AuthenticationResponse authenticationResponse;
+
+        try {
+            authenticationResponse = identityProvider.authenticate(authenticationRequest);
+            final String token = jwtService.generateSignedToken(authenticationResponse);
+            return token;
+        } catch (final IdentityAccessException | JwtException e) {
+            throw new AdministrationException(e.getMessage());
         }
 
-        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();
+    }
+
+    /**
+     * A helper function that generates a prioritized list of IdentityProviders to use to
+     * attempt client authentication.
+     *
+     * Note: This is currently a hard-coded list order consisting of:
+     *
+     * - X509IdentityProvider (if available)
+     * - KerberosProvider (if available)
+     * - User-defined IdentityProvider (if available)
+     *
+     * However, in the future it could be entirely user-configurable
+     *
+     * @return a list of providers to use in order to authenticate the client.
+     */
+    private List<IdentityProvider> generateIdentityProviderWaterfall() {
+        List<IdentityProvider> identityProviderWaterfall = new ArrayList<>();
+
+        // if configured with an X509IdentityProvider, add it to the list of providers to try
+        if (x509IdentityProvider != null) {
+            identityProviderWaterfall.add(x509IdentityProvider);
+        }
+
+        // if configured with an KerberosSpnegoIdentityProvider, add it to the end of the list of providers to try
+        if (kerberosSpnegoIdentityProvider != null) {
+            identityProviderWaterfall.add(kerberosSpnegoIdentityProvider);
+        }
+
+        // if configured with custom identity provider, add it to the end of the list of providers to try
+        if (identityProvider != null) {
+            identityProviderWaterfall.add(identityProvider);
+        }
 
+        return identityProviderWaterfall;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/exception/UnauthorizedException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/exception/UnauthorizedException.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/exception/UnauthorizedException.java
new file mode 100644
index 0000000..46e6fc9
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/exception/UnauthorizedException.java
@@ -0,0 +1,74 @@
+/*
+ * 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.exception;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.security.authentication.IdentityProviderUsage;
+
+import java.util.List;
+
+/**
+ * An exception for a convenient way to create a 401 Unauthorized response
+ * using an exception mapper
+ */
+public class UnauthorizedException extends RuntimeException {
+
+    private String[] wwwAuthenticateChallenge;
+
+    public UnauthorizedException() {
+    }
+
+    public UnauthorizedException(String message) {
+        super(message);
+    }
+
+    public UnauthorizedException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public UnauthorizedException(Throwable cause) {
+        super(cause);
+    }
+
+    public UnauthorizedException withAuthenticateChallenge(IdentityProviderUsage.AuthType authType) {
+        wwwAuthenticateChallenge = new String[] { authType.getHttpAuthScheme() };
+        return this;
+    }
+
+    public UnauthorizedException withAuthenticateChallenge(List<IdentityProviderUsage.AuthType> authTypes) {
+        wwwAuthenticateChallenge = new String[authTypes.size()];
+        for (int i = 0; i < authTypes.size(); i++) {
+            wwwAuthenticateChallenge[i] = authTypes.get(i).getHttpAuthScheme();
+        }
+        return this;
+    }
+
+    public UnauthorizedException withAuthenticateChallenge(String authType) {
+        wwwAuthenticateChallenge = new String[] { authType };
+        return this;
+    }
+
+    public UnauthorizedException withAuthenticateChallenge(String[] authTypes) {
+        wwwAuthenticateChallenge = authTypes;
+        return this;
+    }
+
+    public String getWwwAuthenticateChallenge() {
+        return StringUtils.join(wwwAuthenticateChallenge, ",");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/NotAllowedExceptionMapper.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/NotAllowedExceptionMapper.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/NotAllowedExceptionMapper.java
new file mode 100644
index 0000000..9237735
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/NotAllowedExceptionMapper.java
@@ -0,0 +1,46 @@
+/*
+ * 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.mapper;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.ws.rs.NotAllowedException;
+import javax.ws.rs.core.Response;
+import javax.ws.rs.core.Response.Status;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+/**
+ * Maps exceptions into client responses.
+ */
+@Component
+@Provider
+public class NotAllowedExceptionMapper implements ExceptionMapper<NotAllowedException> {
+
+    private static final Logger logger = LoggerFactory.getLogger(NotAllowedExceptionMapper.class);
+
+    @Override
+    public Response toResponse(NotAllowedException exception) {
+        logger.info(String.format("%s. Returning %s response.", exception, Status.METHOD_NOT_ALLOWED));
+        logger.debug(StringUtils.EMPTY, exception);
+        return Response.status(Status.METHOD_NOT_ALLOWED).entity(exception.getMessage()).type("text/plain").build();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/UnauthorizedExceptionMapper.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/UnauthorizedExceptionMapper.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/UnauthorizedExceptionMapper.java
new file mode 100644
index 0000000..1c67e94
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/UnauthorizedExceptionMapper.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.web.mapper;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.web.exception.UnauthorizedException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Component;
+
+import javax.ws.rs.core.Response;
+import javax.ws.rs.ext.ExceptionMapper;
+import javax.ws.rs.ext.Provider;
+
+/**
+ * Maps Unauthorized exceptions into client responses that set the WWW-Authenticate header
+ * with a list of challenges (i.e., acceptable auth scheme types).
+ */
+@Component
+@Provider
+public class UnauthorizedExceptionMapper implements ExceptionMapper<UnauthorizedException> {
+
+    private static final Logger logger = LoggerFactory.getLogger(UnauthorizedExceptionMapper.class);
+
+    private static final String AUTHENTICATION_CHALLENGE_HEADER_NAME = "WWW-Authenticate";
+
+    @Override
+    public Response toResponse(UnauthorizedException exception) {
+
+        logger.info("{}. Returning {} response.", exception, Response.Status.UNAUTHORIZED);
+        logger.debug(StringUtils.EMPTY, exception);
+
+        final Response.ResponseBuilder response = Response.status(Response.Status.UNAUTHORIZED);
+        if (exception.getWwwAuthenticateChallenge() != null) {
+            response.header(AUTHENTICATION_CHALLENGE_HEADER_NAME, exception.getWwwAuthenticateChallenge());
+        }
+        response.entity(exception.getMessage()).type("text/plain");
+        return response.build();
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/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 16133ca..3f428ee 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
@@ -46,17 +46,21 @@ import org.springframework.security.web.authentication.AnonymousAuthenticationFi
 public class NiFiRegistrySecurityConfig extends WebSecurityConfigurerAdapter {
     private static final Logger logger = LoggerFactory.getLogger(NiFiRegistrySecurityConfig.class);
 
-    @Autowired private NiFiRegistryProperties properties;
+    @Autowired
+    private NiFiRegistryProperties properties;
 
-    @Autowired private Authorizer authorizer;
+    @Autowired
+    private Authorizer authorizer;
 
     private AnonymousIdentityFilter anonymousAuthenticationFilter = new AnonymousIdentityFilter();
 
-    @Autowired private X509IdentityProvider x509IdentityProvider;
+    @Autowired
+    private X509IdentityProvider x509IdentityProvider;
     private IdentityFilter x509AuthenticationFilter;
     private IdentityAuthenticationProvider x509AuthenticationProvider;
 
-    @Autowired private JwtIdentityProvider jwtIdentityProvider;
+    @Autowired
+    private JwtIdentityProvider jwtIdentityProvider;
     private IdentityFilter jwtAuthenticationFilter;
     private IdentityAuthenticationProvider jwtAuthenticationProvider;
 
@@ -67,7 +71,7 @@ public class NiFiRegistrySecurityConfig extends WebSecurityConfigurerAdapter {
     @Override
     public void configure(WebSecurity webSecurity) throws Exception {
         // allow any client to access the endpoint for logging in to generate an access token
-        webSecurity.ignoring().antMatchers( "/access/token/*");
+        webSecurity.ignoring().antMatchers( "/access/token/**");
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/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
index 7c2b27f..8367eec 100644
--- 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
@@ -27,7 +27,6 @@ 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;
@@ -39,7 +38,6 @@ 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}.
@@ -76,42 +74,26 @@ public class IdentityAuthenticationFilter extends AbstractAuthenticationProcessi
 
         // 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
+            throw new InvalidAuthenticationException("Authentication of user identity claim is only avaialble when running a securely.");
         }
 
         AuthenticationRequest authenticationRequest = identityProvider.extractCredentials(httpServletRequest);
         if (authenticationRequest == null) {
-            //return null;
-            return new ContinueFilterChainAuthentication(); // see successfulAuthentication for why we do this
+            throw new InvalidAuthenticationException("User credentials not found in httpServletRequest by " + identityProvider.getClass().getSimpleName());
         }
         Authentication authentication = new AuthenticationRequestToken(authenticationRequest, identityProvider.getClass(), httpServletRequest.getRemoteAddr());
-        Authentication authenticationResult = getAuthenticationManager().authenticate(authentication); // See IdentityProviderAuthenticationProvider for authentication impl.
+        Authentication authenticationResult = getAuthenticationManager().authenticate(authentication); // See IdentityAuthenticationProvider for authentication impl.
         if (authenticationResult == null) {
-            return new ContinueFilterChainAuthentication(); // see successfulAuthentication for why we do this
-        } else {
-            return authenticationResult;
+            throw new InvalidAuthenticationException("User credentials not authenticated by " + identityProvider.getClass().getSimpleName());
         }
+
+        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);
@@ -163,41 +145,4 @@ public class IdentityAuthenticationFilter extends AbstractAuthenticationProcessi
         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/58925377/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
index 9592c2b..d3f12c9 100644
--- 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
@@ -54,14 +54,23 @@ public class JwtIdentityProvider extends BearerAuthIdentityProvider implements I
 
     @Override
     public AuthenticationResponse authenticate(AuthenticationRequest authenticationRequest) throws InvalidCredentialsException, IdentityAccessException {
+
+        if (authenticationRequest == null) {
+            logger.info("Cannot authenticate null authenticationRequest, returning null.");
+            return null;
+        }
+
+        final Object credentials = authenticationRequest.getCredentials();
+        String jwtAuthToken = credentials != null && credentials instanceof String ? (String) credentials : null;
+
+        if (credentials == null) {
+            logger.info("JWT not found in authenticationRequest credentials, returning null.");
+            return null;
+        }
+
         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);
         }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/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 4401a15..d47b301 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,7 +28,6 @@ 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;
@@ -104,7 +103,7 @@ public class JwtService {
                     return key.getKey().getBytes(StandardCharsets.UTF_8);
                 }
             }).parseClaimsJws(base64EncodedToken);
-        } catch (final MalformedJwtException | UnsupportedJwtException | SignatureException | ExpiredJwtException | IllegalArgumentException | AdministrationException e) {
+        } catch (final MalformedJwtException | UnsupportedJwtException | SignatureException | ExpiredJwtException | IllegalArgumentException e) {
             // TODO: Exercise all exceptions to ensure none leak key material to logs
             final String errorMessage = "Unable to validate the access token.";
             throw new JwtException(errorMessage, e);
@@ -112,59 +111,66 @@ public class JwtService {
     }
 
     /**
-     * Generates a signed JWT token from the provided (Spring Security) login authentication token.
+     * Generates a signed JWT token from the provided IdentityProvider AuthenticationResponse
      *
-     * @param authenticationResponse an instance of the Spring Security token after login credentials have been verified against the respective information source
+     * @param authenticationResponse an instance issued by an IdentityProvider after identity claim has been verified as authentic
      * @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 AuthenticationResponse authenticationResponse) throws JwtException {
         if (authenticationResponse == null) {
-            throw new IllegalArgumentException("Cannot generate a JWT for a null authentication token");
+            throw new IllegalArgumentException("Cannot generate a JWT for a null authenticationResponse");
         }
 
-        // Set expiration from the token
-        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();
+        return generateSignedToken(
+                authenticationResponse.getIdentity(),
+                authenticationResponse.getUsername(),
+                authenticationResponse.getIssuer(),
+                authenticationResponse.getIssuer(),
+                authenticationResponse.getExpiration());
+    }
+
+    public String generateSignedToken(String identity, String preferredUsername, String issuer, String audience, long expirationMillis) throws JwtException {
 
-        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 " + authenticationResponse.getIssuer();
+        if (identity == null || StringUtils.isEmpty(identity)) {
+            String errorMessage = "Cannot generate a JWT for a token with an empty identity";
+            errorMessage = issuer != null ? errorMessage + " issued by " + issuer + "." : ".";
             logger.error(errorMessage);
-            throw new JwtException(errorMessage);
+            throw new IllegalArgumentException(errorMessage);
         }
 
-        // Create a JWT with the specified authentication
-        final String identity = principal.toString();
-        final String username = authenticationResponse.getUsername();
+        // Compute expiration
+        final Calendar now = Calendar.getInstance();
+        long expirationMillisRelativeToNow = validateTokenExpiration(expirationMillis, identity);
+        long expirationMillisSinceEpoch = now.getTimeInMillis() + expirationMillisRelativeToNow;
+        final Calendar expiration = new Calendar.Builder().setInstant(expirationMillisSinceEpoch).build();
 
         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 " + describe(authenticationResponse));
+            //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(authenticationResponse.getIssuer())
-                    .setAudience(authenticationResponse.getIssuer())
-                    .claim(USERNAME_CLAIM, username)
+                    .setIssuer(issuer)
+                    .setAudience(audience)
+                    .claim(USERNAME_CLAIM, preferredUsername)
                     .claim(KEY_ID_CLAIM, key.getId())
                     .setIssuedAt(now.getTime())
                     .setExpiration(expiration.getTime())
                     .signWith(SIGNATURE_ALGORITHM, keyBytes).compact();
-        } catch (NullPointerException | AdministrationException e) {
+        } catch (NullPointerException e) {
             final String errorMessage = "Could not retrieve the signing key for JWT for " + identity;
             logger.error(errorMessage, e);
             throw new JwtException(errorMessage, e);
         }
+
     }
 
-    private long validateTokenExpiration(long proposedTokenExpiration, String identity) {
+    private static long validateTokenExpiration(long proposedTokenExpiration, String identity) {
         final long maxExpiration = TimeUnit.MILLISECONDS.convert(12, TimeUnit.HOURS);
         final long minExpiration = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
 

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/58925377/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosIdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosIdentityProvider.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosIdentityProvider.java
new file mode 100644
index 0000000..5e6e7bb
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/kerberos/KerberosIdentityProvider.java
@@ -0,0 +1,111 @@
+/*
+ * 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.kerberos;
+
+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.BasicAuthIdentityProvider;
+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.exception.SecurityProviderCreationException;
+import org.apache.nifi.registry.security.exception.SecurityProviderDestructionException;
+import org.apache.nifi.registry.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.security.authentication.UsernamePasswordAuthenticationToken;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.AuthenticationException;
+import org.springframework.security.kerberos.authentication.KerberosAuthenticationProvider;
+import org.springframework.security.kerberos.authentication.sun.SunJaasKerberosClient;
+
+import java.util.concurrent.TimeUnit;
+
+public class KerberosIdentityProvider extends BasicAuthIdentityProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(KerberosIdentityProvider.class);
+    private static final String issuer = KerberosIdentityProvider.class.getSimpleName();
+    private static final String default_expiration = "12 hours";
+
+    private KerberosAuthenticationProvider provider;
+
+    private long expiration;
+
+    @Override
+    public void onConfigured(IdentityProviderConfigurationContext configurationContext) throws SecurityProviderCreationException {
+
+        String rawDebug = configurationContext.getProperty("Enable Debug");
+        boolean enableDebug = (rawDebug != null && rawDebug.equalsIgnoreCase("true"));
+
+        String rawExpiration = configurationContext.getProperty("Authentication Expiration");
+        if (StringUtils.isBlank(rawExpiration)) {
+            rawExpiration = default_expiration;
+            logger.info("No Authentication Expiration specified, defaulting to " + default_expiration);
+        }
+
+        try {
+            expiration = FormatUtils.getTimeDuration(rawExpiration, TimeUnit.MILLISECONDS);
+        } catch (final IllegalArgumentException iae) {
+            throw new SecurityProviderCreationException(
+                    String.format("The Expiration Duration '%s' is not a valid time duration", rawExpiration));
+        }
+
+        provider = new KerberosAuthenticationProvider();
+        SunJaasKerberosClient client = new SunJaasKerberosClient();
+        client.setDebug(enableDebug);
+        provider.setKerberosClient(client);
+        provider.setUserDetailsService(new KerberosUserDetailsService());
+
+    }
+
+    @Override
+    public AuthenticationResponse authenticate(AuthenticationRequest authenticationRequest) throws InvalidCredentialsException, IdentityAccessException {
+
+        if (provider == null) {
+            throw new IdentityAccessException("The Kerberos authentication provider is not initialized.");
+        }
+
+        try {
+            // perform the authentication
+            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);
+            logger.debug("Created authentication token " + token.toString());
+
+            final Authentication authentication = provider.authenticate(token);
+            logger.debug("Ran provider.authenticate(token) and returned authentication for " +
+                    "principal={} with name={} and isAuthenticated={}",
+                    authentication.getPrincipal(),
+                    authentication.getName(),
+                    authentication.isAuthenticated());
+
+            return new AuthenticationResponse(authentication.getName(), username, expiration, issuer);
+        } catch (final AuthenticationException e) {
+            throw new InvalidCredentialsException(e.getMessage(), e);
+        }
+
+    }
+
+    @Override
+    public void preDestruction() throws SecurityProviderDestructionException {
+
+    }
+}