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/07 18:50:17 UTC

[01/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Repository: nifi-registry
Updated Branches:
  refs/heads/master a43e81fed -> 90f36dd22


http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/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
new file mode 100644
index 0000000..5f5af59
--- /dev/null
+++ b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureLdapIT.java
@@ -0,0 +1,230 @@
+/*
+ * 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.SecureLdapTestApiApplication;
+import org.apache.nifi.registry.extension.ExtensionManager;
+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;
+import org.skyscreamer.jsonassert.JSONAssert;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.boot.test.context.TestConfiguration;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.DependsOn;
+import org.springframework.context.annotation.Import;
+import org.springframework.context.annotation.Primary;
+import org.springframework.context.annotation.Profile;
+import org.springframework.test.context.jdbc.Sql;
+import org.springframework.test.context.junit4.SpringRunner;
+
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.core.Form;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import static org.junit.Assert.assertEquals;
+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
+ * - 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(
+        classes = SecureLdapTestApiApplication.class,
+        webEnvironment = SpringBootTest.WebEnvironment.RANDOM_PORT,
+        properties = "spring.profiles.include=ITSecureLdap")
+@Import(SecureITClientConfiguration.class)
+@Sql(executionPhase = Sql.ExecutionPhase.BEFORE_TEST_METHOD, scripts = "classpath:db/clearDB.sql")
+public class SecureLdapIT extends IntegrationTestBase {
+
+    @TestConfiguration
+    @Profile("ITSecureLdap")
+    public static class LdapTestConfiguration {
+
+        static AuthorizerFactory af;
+
+        @Primary
+        @Bean
+        @DependsOn({"directoryServer"}) // Can't load LdapUserGroupProvider until the embedded LDAP server, which creates the "directoryServer" bean, is running
+        public static Authorizer getAuthorizer(@Autowired NiFiRegistryProperties properties, ExtensionManager extensionManager) {
+            if (af == null) {
+                af = new AuthorizerFactory(properties, extensionManager);
+            }
+            return af.getAuthorizer();
+        }
+
+    }
+
+    private String authToken;
+
+    @Before
+    public void generateAuthToken() {
+        final Form form = new Form()
+                .param("username", "nifiadmin")
+                .param("password", "password");
+        final String token = client
+                .target(createURL("access/token"))
+                .request()
+                .post(Entity.form(form), String.class);
+        authToken = token;
+    }
+
+    @Test
+    public void testTokenGeneration() 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 LDAP authentication
+        String expectedJwtPayloadJson = "{" +
+                "\"sub\":\"nobel\"," +
+                "\"preferred_username\":\"nobel\"," +
+                "\"iss\":\"LdapIdentityProvider\"," +
+                "\"aud\":\"LdapIdentityProvider\"" +
+                "}";
+        String expectedAccessStatusJson = "{" +
+                "\"identity\":\"nobel\"," +
+                "\"status\":\"ACTIVE\"" +
+                "}";
+
+        // When: the /access/token endpoint is queried
+        final Form form = new Form()
+                .param("username", "nobel")
+                .param("password", "password");
+        final Response tokenResponse = client
+                .target(createURL("access/token"))
+                .request()
+                .post(Entity.form(form), Response.class);
+
+        // Then: the server returns 200 OK with an access token
+        assertEquals(201, tokenResponse.getStatus());
+        String token = tokenResponse.readEntity(String.class);
+        assertTrue(StringUtils.isNotEmpty(token));
+        String[] jwtParts = token.split("\\.");
+        assertEquals(3, jwtParts.length);
+        String jwtPayload = new String(Base64.decodeBase64(jwtParts[1]), "UTF-8");
+        JSONAssert.assertEquals(expectedJwtPayloadJson, jwtPayload, false);
+
+        // When: the token is returned in the Authorization header
+        final Response accessResponse = client
+                .target(createURL("access"))
+                .request()
+                .header("Authorization", "Bearer " + token)
+                .get(Response.class);
+
+        // Then: the server acknowledges the client has access
+        assertEquals(200, accessResponse.getStatus());
+        String accessStatus = accessResponse.readEntity(String.class);
+        JSONAssert.assertEquals(expectedAccessStatusJson, accessStatus, false);
+
+    }
+
+    @Test
+    public void testUsers() throws Exception {
+
+        // Given: the client and server have been configured correctly for LDAP authentication
+        String expectedJson = "[" +
+                "{\"identity\":\"nifiadmin\",\"userGroups\":[]}," +
+                "{\"identity\":\"euler\",\"userGroups\":[{\"identity\":\"mathematicians\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"euclid\",\"userGroups\":[{\"identity\":\"mathematicians\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"boyle\",\"userGroups\":[{\"identity\":\"chemists\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"newton\",\"userGroups\":[{\"identity\":\"scientists\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"riemann\",\"userGroups\":[{\"identity\":\"mathematicians\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"gauss\",\"userGroups\":[{\"identity\":\"mathematicians\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"galileo\",\"userGroups\":[{\"identity\":\"scientists\"},{\"identity\":\"italians\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"nobel\",\"userGroups\":[{\"identity\":\"chemists\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"pasteur\",\"userGroups\":[{\"identity\":\"chemists\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"tesla\",\"userGroups\":[{\"identity\":\"scientists\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"nogroup\",\"userGroups\":[],\"accessPolicies\":[]}," +
+                "{\"identity\":\"einstein\",\"userGroups\":[{\"identity\":\"scientists\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"curie\",\"userGroups\":[{\"identity\":\"chemists\"}],\"accessPolicies\":[]}]";
+
+        // When: the /tenants/users endpoint is queried
+        final String usersJson = client
+                .target(createURL("tenants/users"))
+                .request()
+                .header("Authorization", "Bearer " + authToken)
+                .get(String.class);
+
+        // Then: the server returns a list of all users (see test-ldap-data.ldif)
+        JSONAssert.assertEquals(expectedJson, usersJson, false);
+    }
+
+    @Test
+    public void testUserGroups() throws Exception {
+
+        // Given: the client and server have been configured correctly for LDAP authentication
+        String expectedJson = "[" +
+                "{\"identity\":\"chemists\",\"users\":[{\"identity\":\"pasteur\"},{\"identity\":\"boyle\"},{\"identity\":\"curie\"},{\"identity\":\"nobel\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"mathematicians\",\"users\":[{\"identity\":\"gauss\"},{\"identity\":\"euclid\"},{\"identity\":\"riemann\"},{\"identity\":\"euler\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"scientists\",\"users\":[{\"identity\":\"einstein\"},{\"identity\":\"tesla\"},{\"identity\":\"newton\"},{\"identity\":\"galileo\"}],\"accessPolicies\":[]}," +
+                "{\"identity\":\"italians\",\"users\":[{\"identity\":\"galileo\"}],\"accessPolicies\":[]}]";
+
+        // When: the /tenants/users endpoint is queried
+        final String groupsJson = client
+                .target(createURL("tenants/user-groups"))
+                .request()
+                .header("Authorization", "Bearer " + authToken)
+                .get(String.class);
+
+        // Then: the server returns a list of all users (see test-ldap-data.ldif)
+        JSONAssert.assertEquals(expectedJson, groupsJson, false);
+    }
+
+
+    public void testCreateTenantFails() throws Exception {
+
+        // Given: the server has been configured with the LdapUserGroupProvider, which is non-configurable,
+        //   and: the client wants to create a tenant
+        Tenant tenant = new Tenant();
+        tenant.setIdentity("new_tenant");
+
+        // When: the POST /tenants/users endpoint is accessed
+        final Response createUserResponse = client
+                .target(createURL("tenants/users"))
+                .request()
+                .header("Authorization", "Bearer " + authToken)
+                .post(Entity.entity(tenant, MediaType.APPLICATION_JSON_TYPE), Response.class);
+
+        // Then: an error is returned
+        assertEquals(405, createUserResponse.getStatus());
+
+        // When: the POST /tenants/users endpoint is accessed
+        final Response createUserGroupResponse = client
+                .target(createURL("tenants/user-groups"))
+                .request()
+                .header("Authorization", "Bearer " + authToken)
+                .post(Entity.entity(tenant, MediaType.APPLICATION_JSON_TYPE), Response.class);
+
+        // Then: an error is returned because the UserGroupProvider is non-configurable
+        assertEquals(405, createUserGroupResponse.getStatus());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/UnsecuredITBase.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/UnsecuredITBase.java b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/UnsecuredITBase.java
index e546511..a0c981b 100644
--- a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/UnsecuredITBase.java
+++ b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/UnsecuredITBase.java
@@ -16,7 +16,7 @@
  */
 package org.apache.nifi.registry.web.api;
 
-import org.apache.nifi.registry.NiFiRegistryApiTestApplication;
+import org.apache.nifi.registry.NiFiRegistryTestApiApplication;
 import org.junit.runner.RunWith;
 import org.springframework.boot.test.context.SpringBootTest;
 import org.springframework.test.context.jdbc.Sql;
@@ -31,7 +31,7 @@ import org.springframework.test.context.junit4.SpringRunner;
  */
 @RunWith(SpringRunner.class)
 @SpringBootTest(
-        classes = NiFiRegistryApiTestApplication.class,
+        classes = NiFiRegistryTestApiApplication.class,
         webEnvironment = SpringBootTest.WebEnvironment.RANDOM_PORT,
         properties = "spring.profiles.include=ITUnsecured")
 @Sql(executionPhase = Sql.ExecutionPhase.BEFORE_TEST_METHOD, scripts = "classpath:db/clearDB.sql")

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/test/resources/application-ITSecureLdap.properties
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/application-ITSecureLdap.properties b/nifi-registry-web-api/src/test/resources/application-ITSecureLdap.properties
new file mode 100644
index 0000000..ffcc43e
--- /dev/null
+++ b/nifi-registry-web-api/src/test/resources/application-ITSecureLdap.properties
@@ -0,0 +1,48 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+# 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-ldap/nifi-registry.properties
+nifi.registry.client.properties.file: src/test/resources/conf/secure-ldap/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
+
+# Embedded LDAP Config
+spring.ldap.embedded.base-dn: dc=example,dc=com
+spring.ldap.embedded.credential.username: cn=read-only-admin,dc=example,dc=com
+spring.ldap.embedded.credential.password: password
+spring.ldap.embedded.ldif: classpath:conf/secure-ldap/test-ldap-data.ldif
+spring.ldap.embedded.port: 8389
+spring.ldap.embedded.validation.enabled: false
+
+# Additional Logging Config
+logging.level.org.springframework.security.ldap: DEBUG
+logging.level.org.springframework.ldap: DEBUG
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/test/resources/application.properties
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/application.properties b/nifi-registry-web-api/src/test/resources/application.properties
index 9cc9f51..f63f89d 100644
--- a/nifi-registry-web-api/src/test/resources/application.properties
+++ b/nifi-registry-web-api/src/test/resources/application.properties
@@ -24,8 +24,8 @@
 #logging.level.org.springframework.context.annotation: DEBUG
 #logging.level.org.springframework.web: DEBUG
 
-#spring.jpa.show-sql=true
-spring.jpa.hibernate.ddl-auto=validate
-spring.jpa.properties.hibernate.dialect = org.hibernate.dialect.H2Dialect
+# These properties should match the defaultProperties hardcoded in NiFiRegistryApiApplication.configure()
+spring.jpa.hibernate.ddl-auto = none
+spring.jpa.hibernate.naming.physical-strategy = org.hibernate.boot.model.naming.PhysicalNamingStrategyStandardImpl
 
-spring.datasource.url = jdbc:h2:mem:test
\ No newline at end of file
+spring.jpa.show-sql = true
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/test/resources/conf/secure-file/authorizers.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/conf/secure-file/authorizers.xml b/nifi-registry-web-api/src/test/resources/conf/secure-file/authorizers.xml
index 70d75b1..1f92793 100644
--- a/nifi-registry-web-api/src/test/resources/conf/secure-file/authorizers.xml
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-file/authorizers.xml
@@ -44,7 +44,7 @@
     -->
     <userGroupProvider>
         <identifier>file-user-group-provider</identifier>
-        <class>org.apache.nifi.registry.authorization.file.FileUserGroupProvider</class>
+        <class>org.apache.nifi.registry.security.authorization.file.FileUserGroupProvider</class>
         <property name="Users File">./target/test-classes/conf/secure-file/users.xml</property>
         <property name="Initial User Identity 1">CN=user1, OU=nifi</property>
     </userGroupProvider>
@@ -118,7 +118,7 @@
     -->
     <accessPolicyProvider>
         <identifier>file-access-policy-provider</identifier>
-        <class>org.apache.nifi.registry.authorization.file.FileAccessPolicyProvider</class>
+        <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-file/authorizations.xml</property>
         <property name="Initial Admin Identity">CN=user1, OU=nifi</property>
@@ -136,7 +136,7 @@
     -->
     <authorizer>
         <identifier>managed-authorizer</identifier>
-        <class>org.apache.nifi.registry.authorization.StandardManagedAuthorizer</class>
+        <class>org.apache.nifi.registry.security.authorization.StandardManagedAuthorizer</class>
         <property name="Access Policy Provider">file-access-policy-provider</property>
     </authorizer>
 

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/test/resources/conf/secure-file/nifi-registry.properties
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/conf/secure-file/nifi-registry.properties b/nifi-registry-web-api/src/test/resources/conf/secure-file/nifi-registry.properties
index 65271c6..408f44d 100644
--- a/nifi-registry-web-api/src/test/resources/conf/secure-file/nifi-registry.properties
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-file/nifi-registry.properties
@@ -28,6 +28,3 @@ nifi.registry.security.authorizer=managed-authorizer
 
 # providers properties #
 nifi.registry.providers.configuration.file=./target/test-classes/conf/providers.xml
-
-# database properties
-nifi.registry.db.url.append=;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/test/resources/conf/secure-ldap/authorizers.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/conf/secure-ldap/authorizers.xml b/nifi-registry-web-api/src/test/resources/conf/secure-ldap/authorizers.xml
new file mode 100644
index 0000000..ca472cc
--- /dev/null
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-ldap/authorizers.xml
@@ -0,0 +1,243 @@
+<?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).
+    -->
+    <!-- To enable the file-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>file-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.authorization.file.FileUserGroupProvider</class>
+        <property name="Users File">./target/test-classes/conf/secure-file/users.xml</property>
+        <property name="Initial User Identity 1">CN=user1, OU=nifi</property>
+    </userGroupProvider>
+    To enable the file-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
+        The LdapUserGroupProvider will retrieve users and groups from an LDAP server. The users and groups
+        are not configurable.
+
+        'Authentication Strategy' - How the connection to the LDAP server is authenticated. Possible
+            values are ANONYMOUS, SIMPLE, LDAPS, or START_TLS.
+
+        'Manager DN' - The DN of the manager that is used to bind to the LDAP server to search for users.
+        'Manager Password' - The password of the manager that is used to bind to the LDAP server to
+            search for users.
+
+        'TLS - Keystore' - Path to the Keystore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Keystore Password' - Password for the Keystore that is used when connecting to LDAP
+            using LDAPS or START_TLS.
+        'TLS - Keystore Type' - Type of the Keystore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Truststore' - Path to the Truststore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Password' - Password for the Truststore that is used when connecting to
+            LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Type' - Type of the Truststore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Client Auth' - Client authentication policy when connecting to LDAP using LDAPS or START_TLS.
+            Possible values are REQUIRED, WANT, NONE.
+        'TLS - Protocol' - Protocol to use when connecting to LDAP using LDAPS or START_TLS. (i.e. TLS,
+            TLSv1.1, TLSv1.2, etc).
+        'TLS - Shutdown Gracefully' - Specifies whether the TLS should be shut down gracefully
+            before the target context is closed. Defaults to false.
+
+        'Referral Strategy' - Strategy for handling referrals. Possible values are FOLLOW, IGNORE, THROW.
+        'Connect Timeout' - Duration of connect timeout. (i.e. 10 secs).
+        'Read Timeout' - Duration of read timeout. (i.e. 10 secs).
+
+        'Url' - Space-separated list of URLs of the LDAP servers (i.e. ldap://<hostname>:<port>).
+        'Page Size' - Sets the page size when retrieving users and groups. If not specified, no paging is performed.
+        'Sync Interval' - Duration of time between syncing users and groups. (i.e. 30 mins).
+
+        'User Search Base' - Base DN for searching for users (i.e. ou=users,o=nifi). Required to search users.
+        'User Object Class' - Object class for identifying users (i.e. person). Required if searching users.
+        'User Search Scope' - Search scope for searching users (ONE_LEVEL, OBJECT, or SUBTREE). Required if searching users.
+        'User Search Filter' - Filter for searching for users against the 'User Search Base' (i.e. (memberof=cn=team1,ou=groups,o=nifi) ). Optional.
+        'User Identity Attribute' - Attribute to use to extract user identity (i.e. cn). Optional. If not set, the entire DN is used.
+        'User Group Name Attribute' - Attribute to use to define group membership (i.e. memberof). Optional. If not set
+            group membership will not be calculated through the users. Will rely on group membership being defined
+            through 'Group Member Attribute' if set.
+
+        'Group Search Base' - Base DN for searching for groups (i.e. ou=groups,o=nifi). Required to search groups.
+        'Group Object Class' - Object class for identifying groups (i.e. groupOfNames). Required if searching groups.
+        'Group Search Scope' - Search scope for searching groups (ONE_LEVEL, OBJECT, or SUBTREE). Required if searching groups.
+        'Group Search Filter' - Filter for searching for groups against the 'Group Search Base'. Optional.
+        'Group Name Attribute' - Attribute to use to extract group name (i.e. cn). Optional. If not set, the entire DN is used.
+        'Group Member Attribute' - Attribute to use to define group membership (i.e. member). Optional. If not set
+            group membership will not be calculated through the groups. Will rely on group member being defined
+            through 'User Group Name Attribute' if set.
+
+        NOTE: Any identity mapping rules specified in nifi.properties will also be applied to the user identities.
+            Group names are not mapped.
+    -->
+    <userGroupProvider>
+        <identifier>ldap-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.ldap.tenants.LdapUserGroupProvider</class>
+        <property name="Authentication Strategy">SIMPLE</property>
+
+        <property name="Manager DN">cn=read-only-admin,dc=example,dc=com</property>
+        <property name="Manager Password">password</property>
+
+        <!--
+        <property name="TLS - Keystore"></property>
+        <property name="TLS - Keystore Password"></property>
+        <property name="TLS - Keystore Type"></property>
+        <property name="TLS - Truststore"></property>
+        <property name="TLS - Truststore Password"></property>
+        <property name="TLS - Truststore Type"></property>
+        <property name="TLS - Client Auth"></property>
+        <property name="TLS - Protocol"></property>
+        <property name="TLS - Shutdown Gracefully"></property>
+        -->
+
+        <property name="Referral Strategy">FOLLOW</property>
+        <property name="Connect Timeout">10 secs</property>
+        <property name="Read Timeout">10 secs</property>
+
+        <property name="Url">ldap://localhost:8389</property>
+        <!--<property name="Page Size"></property>-->
+        <property name="Sync Interval">30 mins</property>
+
+        <property name="User Search Base">dc=example,dc=com</property>
+        <property name="User Object Class">person</property>
+        <property name="User Search Scope">ONE_LEVEL</property>
+        <property name="User Search Filter">(uid=*)</property>
+        <property name="User Identity Attribute">uid</property>
+        <!--<property name="User Group Name Attribute"></property>-->
+
+        <property name="Group Search Base">dc=example,dc=com</property>
+        <property name="Group Object Class">groupOfUniqueNames</property>
+        <property name="Group Search Scope">ONE_LEVEL</property>
+        <property name="Group Search Filter">(ou=*)</property>
+        <property name="Group Name Attribute">ou</property>
+        <property name="Group Member Attribute">uniqueMember</property>
+    </userGroupProvider>
+
+    <!--
+        The CompositeUserGroupProvider will provide support for retrieving users and groups from multiple sources.
+
+        - User Group Provider [unique key] - The identifier of user group providers to load from. The name of
+            each property must be unique, for example: "User Group Provider A", "User Group Provider B",
+            "User Group Provider C" or "User Group Provider 1", "User Group Provider 2", "User Group Provider 3"
+
+            NOTE: Any identity mapping rules specified in nifi.properties are not applied in this implementation. This behavior
+            would need to be applied by the base implementation.
+    -->
+    <!-- To enable the composite-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>composite-user-group-provider</identifier>
+        <class>org.apache.nifi.authorization.CompositeUserGroupProvider</class>
+        <property name="User Group Provider 1"></property>
+    </userGroupProvider>
+    To enable the composite-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
+        The CompositeConfigurableUserGroupProvider will provide support for retrieving users and groups from multiple sources.
+        Additionally, a single configurable user group provider is required. Users from the configurable user group provider
+        are configurable, however users loaded from one of the User Group Provider [unique key] will not be.
+
+        - Configurable User Group Provider - A configurable user group provider.
+
+        - User Group Provider [unique key] - The identifier of user group providers to load from. The name of
+            each property must be unique, for example: "User Group Provider A", "User Group Provider B",
+            "User Group Provider C" or "User Group Provider 1", "User Group Provider 2", "User Group Provider 3"
+
+            NOTE: Any identity mapping rules specified in nifi.properties are not applied in this implementation. This behavior
+            would need to be applied by the base implementation.
+    -->
+    <!-- To enable the composite-configurable-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>composite-configurable-user-group-provider</identifier>
+        <class>org.apache.nifi.authorization.CompositeConfigurableUserGroupProvider</class>
+        <property name="Configurable User Group Provider">file-user-group-provider</property>
+        <property name="User Group Provider 1"></property>
+    </userGroupProvider>
+    To enable the composite-configurable-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
+        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">ldap-user-group-provider</property>
+        <property name="Authorizations File">./target/test-classes/conf/secure-ldap/authorizations.xml</property>
+        <property name="Initial Admin Identity">nifiadmin</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/90f36dd2/nifi-registry-web-api/src/test/resources/conf/secure-ldap/identity-providers.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/conf/secure-ldap/identity-providers.xml b/nifi-registry-web-api/src/test/resources/conf/secure-ldap/identity-providers.xml
new file mode 100644
index 0000000..280a975
--- /dev/null
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-ldap/identity-providers.xml
@@ -0,0 +1,88 @@
+<?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>
+    <!--
+        Identity Provider for users logging in with username/password against an LDAP server.
+        
+        'Authentication Strategy' - How the connection to the LDAP server is authenticated. Possible
+            values are ANONYMOUS, SIMPLE, LDAPS, or START_TLS.
+        
+        'Manager DN' - The DN of the manager that is used to bind to the LDAP server to search for users.
+        'Manager Password' - The password of the manager that is used to bind to the LDAP server to
+            search for users.
+            
+        'TLS - Keystore' - Path to the Keystore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Keystore Password' - Password for the Keystore that is used when connecting to LDAP
+            using LDAPS or START_TLS.
+        'TLS - Keystore Type' - Type of the Keystore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Truststore' - Path to the Truststore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Password' - Password for the Truststore that is used when connecting to
+            LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Type' - Type of the Truststore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Client Auth' - Client authentication policy when connecting to LDAP using LDAPS or START_TLS.
+            Possible values are REQUIRED, WANT, NONE.
+        'TLS - Protocol' - Protocol to use when connecting to LDAP using LDAPS or START_TLS. (i.e. TLS,
+            TLSv1.1, TLSv1.2, etc).
+        'TLS - Shutdown Gracefully' - Specifies whether the TLS should be shut down gracefully 
+            before the target context is closed. Defaults to false.
+            
+        'Referral Strategy' - Strategy for handling referrals. Possible values are FOLLOW, IGNORE, THROW.
+        'Connect Timeout' - Duration of connect timeout. (i.e. 10 secs).
+        'Read Timeout' - Duration of read timeout. (i.e. 10 secs).
+       
+        'Url' - Space-separated list of URLs of the LDAP servers (i.e. ldap://<hostname>:<port>).
+        'User Search Base' - Base DN for searching for users (i.e. CN=Users,DC=example,DC=com).
+        'User Search Filter' - Filter for searching for users against the 'User Search Base'.
+            (i.e. sAMAccountName={0}). The user specified name is inserted into '{0}'.
+
+        'Identity Strategy' - Strategy to identify users. Possible values are USE_DN and USE_USERNAME.
+            The default functionality if this property is missing is USE_DN in order to retain
+            backward compatibility. USE_DN will use the full DN of the user entry if possible.
+            USE_USERNAME will use the username the user logged in with.
+        '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.
+    -->
+    <provider>
+        <identifier>ldap-identity-provider</identifier>
+        <class>org.apache.nifi.registry.security.ldap.LdapIdentityProvider</class>
+        <property name="Authentication Strategy">SIMPLE</property>
+
+        <property name="Manager DN">cn=read-only-admin,dc=example,dc=com</property>
+        <property name="Manager Password">password</property>
+        
+        <property name="Referral Strategy">FOLLOW</property>
+        <property name="Connect Timeout">10 secs</property>
+        <property name="Read Timeout">10 secs</property>
+
+        <property name="Url">ldap://localhost:8389</property>
+        <property name="User Search Base">dc=example,dc=com</property>
+        <property name="User Search Filter">(uid={0})</property>
+
+        <property name="Identity Strategy">USE_USERNAME</property>
+        <property name="Authentication Expiration">12 hours</property>
+    </provider>
+
+</identityProviders>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/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
new file mode 100644
index 0000000..929e1a7
--- /dev/null
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-ldap/nifi-registry-client.properties
@@ -0,0 +1,25 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# 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
+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/90f36dd2/nifi-registry-web-api/src/test/resources/conf/secure-ldap/nifi-registry.properties
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/conf/secure-ldap/nifi-registry.properties b/nifi-registry-web-api/src/test/resources/conf/secure-ldap/nifi-registry.properties
new file mode 100644
index 0000000..40b2916
--- /dev/null
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-ldap/nifi-registry.properties
@@ -0,0 +1,32 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# 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-ldap/authorizers.xml
+nifi.registry.security.authorizer=managed-authorizer
+nifi.registry.security.identity.providers.configuration.file=./target/test-classes/conf/secure-ldap/identity-providers.xml
+nifi.registry.security.identity.provider=ldap-identity-provider
+
+# providers properties #
+nifi.registry.providers.configuration.file=./target/test-classes/conf/providers.xml

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/test/resources/conf/secure-ldap/test-ldap-data.ldif
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/resources/conf/secure-ldap/test-ldap-data.ldif b/nifi-registry-web-api/src/test/resources/conf/secure-ldap/test-ldap-data.ldif
new file mode 100644
index 0000000..db45689
--- /dev/null
+++ b/nifi-registry-web-api/src/test/resources/conf/secure-ldap/test-ldap-data.ldif
@@ -0,0 +1,261 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# extended LDIF
+#
+# LDAPv3
+# base <dc=example,dc=com> with scope subtree
+# filter: objectclass=*
+# requesting: ALL
+#
+# Adapted from Forum Systems' LDAP Test Server
+#
+
+# example.com
+dn: dc=example,dc=com
+objectClass: top
+objectClass: dcObject
+objectClass: organization
+o: example.com
+dc: example
+
+# read-only-admin, example.com
+dn: cn=read-only-admin,dc=example,dc=com
+sn: Read Only Admin
+cn: read-only-admin
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+
+# nifiadmin, example.com
+dn: uid=nifiadmin,dc=example,dc=com
+sn: nifiadmin
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+uid: nifiadmin
+cn: NiFi Admin
+userPassword: password
+
+# newton, example.com
+dn: uid=newton,dc=example,dc=com
+sn: Newton
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+uid: newton
+cn: Isaac Newton
+userPassword: password
+
+# einstein, example.com
+dn: uid=einstein,dc=example,dc=com
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: Albert Einstein
+sn: Einstein
+uid: einstein
+userPassword: password
+
+# tesla, example.com
+dn: uid=tesla,dc=example,dc=com
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+objectClass: posixAccount
+cn: Nikola Tesla
+sn: Tesla
+uid: tesla
+uidNumber: 88888
+gidNumber: 99999
+homeDirectory: home
+userPassword: password
+
+# galileo, example.com
+dn: uid=galileo,dc=example,dc=com
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: Galileo Galilei
+sn: Galilei
+uid: galileo
+mail: galileo@example.com
+userPassword: password
+
+# euler, example.com
+dn: uid=euler,dc=example,dc=com
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+uid: euler
+sn: Euler
+cn: Leonhard Euler
+userPassword: password
+
+# gauss, example.com
+dn: uid=gauss,dc=example,dc=com
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: Carl Friedrich Gauss
+sn: Gauss
+uid: gauss
+userPassword: password
+
+# riemann, example.com
+dn: uid=riemann,dc=example,dc=com
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: Bernhard Riemann
+sn: Riemann
+uid: riemann
+userPassword: password
+
+# euclid, example.com
+dn: uid=euclid,dc=example,dc=com
+uid: euclid
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: Euclid
+sn: Euclid
+userPassword: password
+
+# curie, example.com
+dn: uid=curie,dc=example,dc=com
+uid: curie
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: Marie Curie
+sn: Curie
+userPassword: password
+
+# nobel, example.com
+dn: uid=nobel,dc=example,dc=com
+uid: nobel
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+sn: Nobel
+cn: Alfred Nobel
+userPassword: password
+
+# boyle, example.com
+dn: uid=boyle,dc=example,dc=com
+uid: boyle
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: Robert Boyle
+sn: Boyle
+telephoneNumber: 999-867-5309
+userPassword: password
+
+# pasteur, example.com
+dn: uid=pasteur,dc=example,dc=com
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+sn: Pasteur
+cn: Louis Pasteur
+uid: pasteur
+telephoneNumber: 602-214-4978
+userPassword: password
+
+# nogroup, example.com
+dn: uid=nogroup,dc=example,dc=com
+uid: nogroup
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: No Group
+sn: Group
+
+# test, example.com
+dn: uid=test,dc=example,dc=com
+objectClass: posixAccount
+objectClass: top
+objectClass: inetOrgPerson
+gidNumber: 0
+givenName: Test
+sn: Test
+displayName: Test
+uid: test
+initials: TS
+homeDirectory: home
+cn: Test
+uidNumber: 24601
+o: Company
+
+# mathematicians, example.com
+dn: ou=mathematicians,dc=example,dc=com
+uniqueMember: uid=euclid,dc=example,dc=com
+uniqueMember: uid=riemann,dc=example,dc=com
+uniqueMember: uid=euler,dc=example,dc=com
+uniqueMember: uid=gauss,dc=example,dc=com
+uniqueMember: uid=test,dc=example,dc=com
+ou: mathematicians
+cn: Mathematicians
+objectClass: groupOfUniqueNames
+objectClass: top
+
+# scientists, example.com
+dn: ou=scientists,dc=example,dc=com
+uniqueMember: uid=einstein,dc=example,dc=com
+uniqueMember: uid=galileo,dc=example,dc=com
+uniqueMember: uid=tesla,dc=example,dc=com
+uniqueMember: uid=newton,dc=example,dc=com
+ou: scientists
+cn: Scientists
+objectClass: groupOfUniqueNames
+objectClass: top
+
+# italians, example.com
+dn: ou=italians,dc=example,dc=com
+uniqueMember: uid=galileo,dc=example,dc=com
+ou: italians
+cn: Italians
+objectClass: groupOfUniqueNames
+objectClass: top
+
+# chemists, example.com
+dn: ou=chemists,dc=example,dc=com
+ou: chemists
+objectClass: groupOfUniqueNames
+objectClass: top
+uniqueMember: uid=curie,dc=example,dc=com
+uniqueMember: uid=boyle,dc=example,dc=com
+uniqueMember: uid=nobel,dc=example,dc=com
+uniqueMember: uid=pasteur,dc=example,dc=com
+cn: Chemists

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8ee2410..f216bcc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -39,7 +39,6 @@
         <module>nifi-registry-resources</module>
         <module>nifi-registry-runtime</module>
         <module>nifi-registry-security-api</module>
-        <module>nifi-registry-security-api-impl</module>
         <module>nifi-registry-security-utils</module>
 	    <module>nifi-registry-framework</module>
         <module>nifi-registry-provider-api</module>
@@ -111,6 +110,7 @@
         <jax.rs.api.version>2.0.1</jax.rs.api.version>
         <jersey.version>2.25.1</jersey.version>
         <spring.boot.version>1.5.7.RELEASE</spring.boot.version>
+        <spring.security.version>4.2.3.RELEASE</spring.security.version>
         <flyway.version>4.2.0</flyway.version>
     </properties>
 


[10/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/util/XmlUtils.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/util/XmlUtils.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/util/XmlUtils.java
new file mode 100644
index 0000000..2caa8fa
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/util/XmlUtils.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.security.util;
+
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamReader;
+import javax.xml.transform.stream.StreamSource;
+import java.io.InputStream;
+
+public class XmlUtils {
+
+    public static XMLStreamReader createSafeReader(InputStream inputStream) throws XMLStreamException {
+        if (inputStream == null) {
+            throw new IllegalArgumentException("The provided input stream cannot be null");
+        }
+        return createSafeReader(new StreamSource(inputStream));
+    }
+
+    public static XMLStreamReader createSafeReader(StreamSource source) throws XMLStreamException {
+        if (source == null) {
+            throw new IllegalArgumentException("The provided source cannot be null");
+        }
+
+        XMLInputFactory xif = XMLInputFactory.newFactory();
+        xif.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, false);
+        xif.setProperty(XMLInputFactory.SUPPORT_DTD, false);
+        return xif.createXMLStreamReader(source);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/AuthorizationService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/AuthorizationService.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/AuthorizationService.java
index 48a7ae3..2567b55 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/AuthorizationService.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/AuthorizationService.java
@@ -16,28 +16,28 @@
  */
 package org.apache.nifi.registry.service;
 
-import org.apache.nifi.registry.authorization.AccessPolicyProvider;
-import org.apache.nifi.registry.authorization.AccessPolicyProviderInitializationContext;
-import org.apache.nifi.registry.authorization.AuthorizableLookup;
-import org.apache.nifi.registry.authorization.AuthorizeAccess;
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.AuthorizerCapabilityDetection;
-import org.apache.nifi.registry.authorization.AuthorizerConfigurationContext;
-import org.apache.nifi.registry.authorization.ConfigurableAccessPolicyProvider;
-import org.apache.nifi.registry.authorization.ConfigurableUserGroupProvider;
-import org.apache.nifi.registry.authorization.Group;
-import org.apache.nifi.registry.authorization.ManagedAuthorizer;
-import org.apache.nifi.registry.authorization.RequestAction;
-import org.apache.nifi.registry.authorization.UserAndGroups;
-import org.apache.nifi.registry.authorization.UserGroupProvider;
-import org.apache.nifi.registry.authorization.UserGroupProviderInitializationContext;
-import org.apache.nifi.registry.authorization.exception.AccessDeniedException;
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerDestructionException;
-import org.apache.nifi.registry.authorization.resource.ResourceFactory;
-import org.apache.nifi.registry.authorization.resource.ResourceType;
-import org.apache.nifi.registry.authorization.user.NiFiUserUtils;
+import org.apache.nifi.registry.security.authorization.AccessPolicyProvider;
+import org.apache.nifi.registry.security.authorization.AccessPolicyProviderInitializationContext;
+import org.apache.nifi.registry.security.authorization.AuthorizableLookup;
+import org.apache.nifi.registry.security.authorization.AuthorizeAccess;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.AuthorizerCapabilityDetection;
+import org.apache.nifi.registry.security.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.registry.security.authorization.ConfigurableAccessPolicyProvider;
+import org.apache.nifi.registry.security.authorization.ConfigurableUserGroupProvider;
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.ManagedAuthorizer;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.UserAndGroups;
+import org.apache.nifi.registry.security.authorization.UserGroupProvider;
+import org.apache.nifi.registry.security.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.registry.security.authorization.exception.AccessDeniedException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.registry.security.authorization.resource.ResourceFactory;
+import org.apache.nifi.registry.security.authorization.resource.ResourceType;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
 import org.apache.nifi.registry.bucket.Bucket;
 import org.apache.nifi.registry.model.authorization.AccessPolicy;
 import org.apache.nifi.registry.model.authorization.AccessPolicySummary;
@@ -105,11 +105,11 @@ public class AuthorizationService {
     public Tenant getTenant(String identifier) {
         this.readLock.lock();
         try {
-            org.apache.nifi.registry.authorization.User user = userGroupProvider.getUser(identifier);
+            org.apache.nifi.registry.security.authorization.User user = userGroupProvider.getUser(identifier);
             if (user != null) {
                 return tenantToDTO(user);
             } else {
-                org.apache.nifi.registry.authorization.Group group = userGroupProvider.getGroup(identifier);
+                org.apache.nifi.registry.security.authorization.Group group = userGroupProvider.getGroup(identifier);
                 return tenantToDTO(group);
             }
         } finally {
@@ -124,7 +124,7 @@ public class AuthorizationService {
         verifyUserGroupProviderIsConfigurable();
         writeLock.lock();
         try {
-            final org.apache.nifi.registry.authorization.User createdUser =
+            final org.apache.nifi.registry.security.authorization.User createdUser =
                 ((ConfigurableUserGroupProvider) userGroupProvider).addUser(userFromDTO(user));
             return userToDTO(createdUser);
         } finally {
@@ -154,7 +154,7 @@ public class AuthorizationService {
         verifyUserGroupProviderIsConfigurable();
         this.writeLock.lock();
         try {
-            final org.apache.nifi.registry.authorization.User updatedUser =
+            final org.apache.nifi.registry.security.authorization.User updatedUser =
                     ((ConfigurableUserGroupProvider) userGroupProvider).updateUser(userFromDTO(user));
             return userToDTO(updatedUser);
         } finally {
@@ -181,7 +181,7 @@ public class AuthorizationService {
         verifyUserGroupProviderIsConfigurable();
         writeLock.lock();
         try {
-            final org.apache.nifi.registry.authorization.Group createdGroup =
+            final org.apache.nifi.registry.security.authorization.Group createdGroup =
                     ((ConfigurableUserGroupProvider) userGroupProvider).addGroup(userGroupFromDTO(userGroup));
             return userGroupToDTO(createdGroup);
         } finally {
@@ -198,18 +198,18 @@ public class AuthorizationService {
         }
     }
 
-    private List<UserGroup> getUserGroupsForUser(String userIdentifier) {
-        this.readLock.lock();
-        try {
-            return userGroupProvider.getGroups()
-                    .stream()
-                    .filter(group -> group.getUsers().contains(userIdentifier))
-                    .map(this::userGroupToDTO)
-                    .collect(Collectors.toList());
-        } finally {
-            this.readLock.unlock();
-        }
-    }
+//    private List<UserGroup> getUserGroupsForUser(String userIdentifier) {
+//        this.readLock.lock();
+//        try {
+//            return userGroupProvider.getGroups()
+//                    .stream()
+//                    .filter(group -> group.getUsers().contains(userIdentifier))
+//                    .map(this::userGroupToDTO)
+//                    .collect(Collectors.toList());
+//        } finally {
+//            this.readLock.unlock();
+//        }
+//    }
 
     public UserGroup getUserGroup(String identifier) {
         this.readLock.lock();
@@ -224,7 +224,7 @@ public class AuthorizationService {
         verifyUserGroupProviderIsConfigurable();
         writeLock.lock();
         try {
-            final org.apache.nifi.registry.authorization.Group updatedGroup =
+            final org.apache.nifi.registry.security.authorization.Group updatedGroup =
                     ((ConfigurableUserGroupProvider) userGroupProvider).updateGroup(userGroupFromDTO(userGroup));
             return userGroupToDTO(updatedGroup);
         } finally {
@@ -251,7 +251,7 @@ public class AuthorizationService {
         verifyAccessPolicyProviderIsConfigurable();
         writeLock.lock();
         try {
-            org.apache.nifi.registry.authorization.AccessPolicy createdAccessPolicy =
+            org.apache.nifi.registry.security.authorization.AccessPolicy createdAccessPolicy =
                     ((ConfigurableAccessPolicyProvider) accessPolicyProvider).addAccessPolicy(accessPolicyFromDTO(accessPolicy));
             return accessPolicyToDTO(createdAccessPolicy);
         } finally {
@@ -336,12 +336,12 @@ public class AuthorizationService {
         writeLock.lock();
         try {
             // Don't allow changing action or resource of existing policy (should only be adding/removing users/groups)
-            org.apache.nifi.registry.authorization.AccessPolicy currentAccessPolicy =
+            org.apache.nifi.registry.security.authorization.AccessPolicy currentAccessPolicy =
                     accessPolicyProvider.getAccessPolicy(accessPolicy.getIdentifier());
             accessPolicy.setResource(currentAccessPolicy.getResource());
             accessPolicy.setAction(currentAccessPolicy.getAction().toString());
 
-            org.apache.nifi.registry.authorization.AccessPolicy updatedAccessPolicy =
+            org.apache.nifi.registry.security.authorization.AccessPolicy updatedAccessPolicy =
                     ((ConfigurableAccessPolicyProvider) accessPolicyProvider).updateAccessPolicy(accessPolicyFromDTO(accessPolicy));
             return accessPolicyToDTO(updatedAccessPolicy);
         } finally {
@@ -419,8 +419,8 @@ public class AuthorizationService {
         }
     }
 
-    private List<org.apache.nifi.registry.authorization.Resource> getAllAuthorizableResources() {
-        final List<org.apache.nifi.registry.authorization.Resource> resources = new ArrayList<>();
+    private List<org.apache.nifi.registry.security.authorization.Resource> getAllAuthorizableResources() {
+        final List<org.apache.nifi.registry.security.authorization.Resource> resources = new ArrayList<>();
         resources.add(ResourceFactory.getPoliciesResource());
         resources.add(ResourceFactory.getTenantResource());
         resources.add(ResourceFactory.getProxyResource());
@@ -436,19 +436,23 @@ public class AuthorizationService {
     }
 
     private org.apache.nifi.registry.model.authorization.User userToDTO(
-            final org.apache.nifi.registry.authorization.User user) {
+            final org.apache.nifi.registry.security.authorization.User user) {
         if (user == null) {
             return null;
         }
         String userIdentifier = user.getIdentifier();
-        Collection<UserGroup> userGroups = getUserGroupsForUser(userIdentifier);
+
+        Collection<Tenant> groupsContainingUser = userGroupProvider.getGroups().stream()
+                .filter(group -> group.getUsers().contains(userIdentifier))
+                .map(AuthorizationService::tenantToDTO)
+                .collect(Collectors.toList());
         Collection<AccessPolicySummary> accessPolicySummaries = getAccessPolicySummariesForUser(userIdentifier);
 
-        return userToDTO(user, userGroups, accessPolicySummaries);
+        return userToDTO(user, groupsContainingUser, accessPolicySummaries);
     }
 
     private org.apache.nifi.registry.model.authorization.UserGroup userGroupToDTO(
-            final org.apache.nifi.registry.authorization.Group userGroup) {
+            final org.apache.nifi.registry.security.authorization.Group userGroup) {
         if (userGroup == null) {
             return null;
         }
@@ -460,7 +464,7 @@ public class AuthorizationService {
     }
 
     private org.apache.nifi.registry.model.authorization.AccessPolicy accessPolicyToDTO(
-            final org.apache.nifi.registry.authorization.AccessPolicy accessPolicy) {
+            final org.apache.nifi.registry.security.authorization.AccessPolicy accessPolicy) {
         if (accessPolicy == null) {
             return null;
         }
@@ -476,7 +480,7 @@ public class AuthorizationService {
     }
 
     private org.apache.nifi.registry.model.authorization.AccessPolicySummary accessPolicyToSummaryDTO(
-            final org.apache.nifi.registry.authorization.AccessPolicy accessPolicy) {
+            final org.apache.nifi.registry.security.authorization.AccessPolicy accessPolicy) {
         if (accessPolicy == null) {
             return null;
         }
@@ -491,7 +495,7 @@ public class AuthorizationService {
         return accessPolicySummaryDTO;
     }
 
-    private static Resource resourceToDTO(org.apache.nifi.registry.authorization.Resource resource) {
+    private static Resource resourceToDTO(org.apache.nifi.registry.security.authorization.Resource resource) {
         if (resource == null) {
             return null;
         }
@@ -501,33 +505,33 @@ public class AuthorizationService {
         return resourceDto;
     }
 
-    private static Tenant tenantToDTO(org.apache.nifi.registry.authorization.User user) {
+    private static Tenant tenantToDTO(org.apache.nifi.registry.security.authorization.User user) {
         if (user == null) {
             return null;
         }
         return new Tenant(user.getIdentifier(), user.getIdentity());
     }
 
-    private static Tenant tenantToDTO(org.apache.nifi.registry.authorization.Group group) {
+    private static Tenant tenantToDTO(org.apache.nifi.registry.security.authorization.Group group) {
         if (group == null) {
             return null;
         }
         return new Tenant(group.getIdentifier(), group.getName());
     }
 
-    private static org.apache.nifi.registry.authorization.User userFromDTO(
+    private static org.apache.nifi.registry.security.authorization.User userFromDTO(
             final org.apache.nifi.registry.model.authorization.User userDTO) {
         if (userDTO == null) {
             return null;
         }
-        return new org.apache.nifi.registry.authorization.User.Builder()
+        return new org.apache.nifi.registry.security.authorization.User.Builder()
                 .identifier(userDTO.getIdentifier() != null ? userDTO.getIdentifier() : UUID.randomUUID().toString())
                 .identity(userDTO.getIdentity())
                 .build();
     }
 
     private static org.apache.nifi.registry.model.authorization.User userToDTO(
-            final org.apache.nifi.registry.authorization.User user,
+            final org.apache.nifi.registry.security.authorization.User user,
             final Collection<? extends Tenant> userGroups,
             final Collection<? extends AccessPolicySummary> accessPolicies) {
 
@@ -540,12 +544,12 @@ public class AuthorizationService {
         return userDTO;
     }
 
-    private static org.apache.nifi.registry.authorization.Group userGroupFromDTO(
+    private static org.apache.nifi.registry.security.authorization.Group userGroupFromDTO(
             final org.apache.nifi.registry.model.authorization.UserGroup userGroupDTO) {
         if (userGroupDTO == null) {
             return null;
         }
-        org.apache.nifi.registry.authorization.Group.Builder groupBuilder = new org.apache.nifi.registry.authorization.Group.Builder()
+        org.apache.nifi.registry.security.authorization.Group.Builder groupBuilder = new org.apache.nifi.registry.security.authorization.Group.Builder()
                 .identifier(userGroupDTO.getIdentifier() != null ? userGroupDTO.getIdentifier() : UUID.randomUUID().toString())
                 .name(userGroupDTO.getIdentity());
         Set<Tenant> users = userGroupDTO.getUsers();
@@ -556,7 +560,7 @@ public class AuthorizationService {
     }
 
     private static org.apache.nifi.registry.model.authorization.UserGroup userGroupToDTO(
-            final org.apache.nifi.registry.authorization.Group userGroup,
+            final org.apache.nifi.registry.security.authorization.Group userGroup,
             final Collection<? extends Tenant> users,
             final Collection<? extends AccessPolicySummary> accessPolicies) {
         if (userGroup == null) {
@@ -568,10 +572,10 @@ public class AuthorizationService {
         return userGroupDTO;
     }
 
-    private static org.apache.nifi.registry.authorization.AccessPolicy accessPolicyFromDTO(
+    private static org.apache.nifi.registry.security.authorization.AccessPolicy accessPolicyFromDTO(
             final org.apache.nifi.registry.model.authorization.AccessPolicy accessPolicyDTO) {
-        org.apache.nifi.registry.authorization.AccessPolicy.Builder accessPolicyBuilder =
-                new org.apache.nifi.registry.authorization.AccessPolicy.Builder()
+        org.apache.nifi.registry.security.authorization.AccessPolicy.Builder accessPolicyBuilder =
+                new org.apache.nifi.registry.security.authorization.AccessPolicy.Builder()
                         .identifier(accessPolicyDTO.getIdentifier() != null ? accessPolicyDTO.getIdentifier() : UUID.randomUUID().toString())
                         .resource(accessPolicyDTO.getResource())
                         .action(RequestAction.valueOfValue(accessPolicyDTO.getAction()));
@@ -590,7 +594,7 @@ public class AuthorizationService {
     }
 
     private static org.apache.nifi.registry.model.authorization.AccessPolicy accessPolicyToDTO(
-            final org.apache.nifi.registry.authorization.AccessPolicy accessPolicy,
+            final org.apache.nifi.registry.security.authorization.AccessPolicy accessPolicy,
             final Collection<? extends Tenant> userGroups,
             final Collection<? extends Tenant> users,
             final Boolean isConfigurable) {
@@ -613,17 +617,17 @@ public class AuthorizationService {
 
         return new AccessPolicyProvider() {
             @Override
-            public Set<org.apache.nifi.registry.authorization.AccessPolicy> getAccessPolicies() throws AuthorizationAccessException {
+            public Set<org.apache.nifi.registry.security.authorization.AccessPolicy> getAccessPolicies() throws AuthorizationAccessException {
                 throw new IllegalStateException(MSG_NON_MANAGED_AUTHORIZER);
             }
 
             @Override
-            public org.apache.nifi.registry.authorization.AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException {
+            public org.apache.nifi.registry.security.authorization.AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException {
                 throw new IllegalStateException(MSG_NON_MANAGED_AUTHORIZER);
             }
 
             @Override
-            public org.apache.nifi.registry.authorization.AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) throws AuthorizationAccessException {
+            public org.apache.nifi.registry.security.authorization.AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) throws AuthorizationAccessException {
                 throw new IllegalStateException(MSG_NON_MANAGED_AUTHORIZER);
             }
 
@@ -631,17 +635,17 @@ public class AuthorizationService {
             public UserGroupProvider getUserGroupProvider() {
                 return new UserGroupProvider() {
                     @Override
-                    public Set<org.apache.nifi.registry.authorization.User> getUsers() throws AuthorizationAccessException {
+                    public Set<org.apache.nifi.registry.security.authorization.User> getUsers() throws AuthorizationAccessException {
                         throw new IllegalStateException(MSG_NON_MANAGED_AUTHORIZER);
                     }
 
                     @Override
-                    public org.apache.nifi.registry.authorization.User getUser(String identifier) throws AuthorizationAccessException {
+                    public org.apache.nifi.registry.security.authorization.User getUser(String identifier) throws AuthorizationAccessException {
                         throw new IllegalStateException(MSG_NON_MANAGED_AUTHORIZER);
                     }
 
                     @Override
-                    public org.apache.nifi.registry.authorization.User getUserByIdentity(String identity) throws AuthorizationAccessException {
+                    public org.apache.nifi.registry.security.authorization.User getUserByIdentity(String identity) throws AuthorizationAccessException {
                         throw new IllegalStateException(MSG_NON_MANAGED_AUTHORIZER);
                     }
 

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/DataModelMapper.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/DataModelMapper.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/DataModelMapper.java
index 383f819..d83f560 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/DataModelMapper.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/service/DataModelMapper.java
@@ -23,8 +23,10 @@ import org.apache.nifi.registry.db.entity.BucketItemEntityType;
 import org.apache.nifi.registry.db.entity.FlowEntity;
 import org.apache.nifi.registry.db.entity.FlowSnapshotEntity;
 import org.apache.nifi.registry.db.entity.FlowSnapshotEntityKey;
+import org.apache.nifi.registry.db.entity.KeyEntity;
 import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
+import org.apache.nifi.registry.security.key.Key;
 
 import java.util.Date;
 import java.util.LinkedHashSet;
@@ -126,4 +128,20 @@ public class DataModelMapper {
         return metadata;
     }
 
+    public static Key map(final KeyEntity keyEntity) {
+        final Key key = new Key();
+        key.setId(keyEntity.getId());
+        key.setIdentity(keyEntity.getTenantIdentity());
+        key.setKey(keyEntity.getKeyValue());
+        return key;
+    }
+
+    public static KeyEntity map(final Key key) {
+        final KeyEntity keyEntity = new KeyEntity();
+        keyEntity.setId(key.getId());
+        keyEntity.setTenantIdentity(key.getIdentity());
+        keyEntity.setKeyValue(key.getKey());
+        return keyEntity;
+    }
+
 }

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/resources/db/migration/V1__Initial.sql
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/resources/db/migration/V1__Initial.sql b/nifi-registry-framework/src/main/resources/db/migration/V1__Initial.sql
index f7640ed..eb71ae7 100644
--- a/nifi-registry-framework/src/main/resources/db/migration/V1__Initial.sql
+++ b/nifi-registry-framework/src/main/resources/db/migration/V1__Initial.sql
@@ -43,4 +43,11 @@ CREATE TABLE FLOW_SNAPSHOT (
     COMMENTS VARCHAR(4096),
     PRIMARY KEY (FLOW_ID, VERSION),
     FOREIGN KEY (FLOW_ID) REFERENCES FLOW(ID)
+);
+
+CREATE TABLE SIGNING_KEY (
+    ID VARCHAR2(50) NOT NULL,
+    TENANT_IDENTITY VARCHAR2(50) NOT NULL UNIQUE,
+    KEY_VALUE VARCHAR2(50) NOT NULL,
+    PRIMARY KEY (ID)
 );
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/xsd/authorizations.xsd
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/xsd/authorizations.xsd b/nifi-registry-framework/src/main/xsd/authorizations.xsd
new file mode 100644
index 0000000..2c8f805
--- /dev/null
+++ b/nifi-registry-framework/src/main/xsd/authorizations.xsd
@@ -0,0 +1,87 @@
+<?xml version="1.0"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema">
+
+    <xs:complexType name="Policy">
+        <xs:sequence>
+            <xs:element name="group" minOccurs="0" maxOccurs="unbounded" >
+                <xs:complexType>
+                    <xs:attribute name="identifier">
+                        <xs:simpleType>
+                            <xs:restriction base="xs:string">
+                                <xs:minLength value="1"/>
+                                <xs:pattern value=".*[^\s].*"/>
+                            </xs:restriction>
+                        </xs:simpleType>
+                    </xs:attribute>
+                </xs:complexType>
+            </xs:element>
+            <xs:element name="user" minOccurs="0" maxOccurs="unbounded" >
+                <xs:complexType>
+                    <xs:attribute name="identifier">
+                        <xs:simpleType>
+                            <xs:restriction base="xs:string">
+                                <xs:minLength value="1"/>
+                                <xs:pattern value=".*[^\s].*"/>
+                            </xs:restriction>
+                        </xs:simpleType>
+                    </xs:attribute>
+                </xs:complexType>
+            </xs:element>
+        </xs:sequence>
+        <xs:attribute name="identifier">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+        <xs:attribute name="resource">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+        <xs:attribute name="action">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:enumeration value="R"/>
+                    <xs:enumeration value="W"/>
+                    <xs:enumeration value="D"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+    </xs:complexType>
+
+    <xs:complexType name="Policies">
+        <xs:sequence>
+            <xs:element name="policy" type="Policy" minOccurs="0" maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+
+    <!-- top-level authorizations element -->
+    <xs:element name="authorizations">
+        <xs:complexType>
+            <xs:sequence>
+                <xs:element name="policies" type="Policies" minOccurs="0" maxOccurs="1" />
+            </xs:sequence>
+        </xs:complexType>
+    </xs:element>
+
+</xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/xsd/tenants.xsd
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/xsd/tenants.xsd b/nifi-registry-framework/src/main/xsd/tenants.xsd
new file mode 100644
index 0000000..c1193c3
--- /dev/null
+++ b/nifi-registry-framework/src/main/xsd/tenants.xsd
@@ -0,0 +1,96 @@
+<?xml version="1.0"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+      http://www.apache.org/licenses/LICENSE-2.0
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema">
+
+    <!-- group -->
+    <xs:complexType name="Group">
+        <xs:sequence>
+            <xs:element name="user" minOccurs="0" maxOccurs="unbounded" >
+                <xs:complexType>
+                    <xs:attribute name="identifier">
+                        <xs:simpleType>
+                            <xs:restriction base="xs:string">
+                                <xs:minLength value="1"/>
+                                <xs:pattern value=".*[^\s].*"/>
+                            </xs:restriction>
+                        </xs:simpleType>
+                    </xs:attribute>
+                </xs:complexType>
+            </xs:element>
+        </xs:sequence>
+        <xs:attribute name="identifier">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+        <xs:attribute name="name">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+    </xs:complexType>
+
+    <!-- groups -->
+    <xs:complexType name="Groups">
+        <xs:sequence>
+            <xs:element name="group" type="Group" minOccurs="0" maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+
+    <!-- user -->
+    <xs:complexType name="User">
+        <xs:attribute name="identifier">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+        <xs:attribute name="identity">
+            <xs:simpleType>
+                <xs:restriction base="xs:string">
+                    <xs:minLength value="1"/>
+                    <xs:pattern value=".*[^\s].*"/>
+                </xs:restriction>
+            </xs:simpleType>
+        </xs:attribute>
+    </xs:complexType>
+
+    <!-- users -->
+    <xs:complexType name="Users">
+        <xs:sequence>
+            <xs:element name="user" type="User" minOccurs="0" maxOccurs="unbounded"/>
+        </xs:sequence>
+    </xs:complexType>
+
+    <!-- top-level authorizations element -->
+    <xs:element name="tenants">
+        <xs:complexType>
+            <xs:sequence>
+                <xs:element name="groups" type="Groups" minOccurs="0" maxOccurs="1" />
+                <xs:element name="users" type="Users" minOccurs="0" maxOccurs="1" />
+            </xs:sequence>
+        </xs:complexType>
+    </xs:element>
+
+</xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/test/groovy/org/apache/nifi/registry/db/DatabaseKeyServiceSpec.groovy
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/groovy/org/apache/nifi/registry/db/DatabaseKeyServiceSpec.groovy b/nifi-registry-framework/src/test/groovy/org/apache/nifi/registry/db/DatabaseKeyServiceSpec.groovy
new file mode 100644
index 0000000..c3cea6a
--- /dev/null
+++ b/nifi-registry-framework/src/test/groovy/org/apache/nifi/registry/db/DatabaseKeyServiceSpec.groovy
@@ -0,0 +1,85 @@
+/*
+ * 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.db
+
+import org.apache.nifi.registry.db.entity.KeyEntity
+import org.apache.nifi.registry.db.repository.KeyRepository
+import org.apache.nifi.registry.security.key.Key
+import spock.lang.Specification
+
+class DatabaseKeyServiceSpec extends Specification {
+
+    def keyRepository = Mock(KeyRepository)
+
+    DatabaseKeyService keyService
+
+    def setup() {
+        keyService = new DatabaseKeyService(keyRepository)
+    }
+
+    def "get key"() {
+
+        given: "a record exists for id=key1"
+        keyRepository.findOne("key1") >> new KeyEntity([id: "key1", tenantIdentity: "user1", keyValue: "keyValue1"])
+
+        when: "getKey is called with an existing id"
+        Key existingKey = keyService.getKey("key1")
+
+        then: "the existing key is returned as model object"
+        with(existingKey) {
+            id == "key1"
+            identity == "user1"
+            key == "keyValue1"
+        }
+
+        when: "getKey is called for a nonexistent id"
+        Key nonexistentKey = keyService.getKey("key2")
+
+        then: "null is returned"
+        nonexistentKey == null
+
+    }
+
+    def "get or create key"() {
+
+        given: "a record exists for identity=user1"
+        keyRepository.findOneByTenantIdentity("user1") >> new KeyEntity([id: "key1", tenantIdentity: "user1", keyValue: "keyValue1"])
+        keyRepository.findOneByTenantIdentity("user2") >> null
+        keyRepository.save(_ as KeyEntity) >> { KeyEntity ke -> new KeyEntity([id: ke.id, tenantIdentity: ke.tenantIdentity, keyValue: ke.keyValue])}
+
+        when: "getOrCreateKey is called with an existing identity"
+        Key existingKey = keyService.getOrCreateKey("user1")
+
+        then: "the existing key is returned as model object"
+        with(existingKey) {
+            id == "key1"
+            identity == "user1"
+            key == "keyValue1"
+        }
+
+        when: "getOrCreateKey is called for a nonexistent identity"
+        Key newKey = keyService.getOrCreateKey("user2")
+
+        then: "a new key is returned"
+        with(newKey) {
+            id != null
+            identity == "user2"
+            key != null
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/test/groovy/org/apache/nifi/registry/service/AuthorizationServiceSpec.groovy
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/groovy/org/apache/nifi/registry/service/AuthorizationServiceSpec.groovy b/nifi-registry-framework/src/test/groovy/org/apache/nifi/registry/service/AuthorizationServiceSpec.groovy
index 3e99483..93fd452 100644
--- a/nifi-registry-framework/src/test/groovy/org/apache/nifi/registry/service/AuthorizationServiceSpec.groovy
+++ b/nifi-registry-framework/src/test/groovy/org/apache/nifi/registry/service/AuthorizationServiceSpec.groovy
@@ -16,17 +16,17 @@
  */
 package org.apache.nifi.registry.service
 
-import org.apache.nifi.registry.authorization.AccessPolicy as AuthAccessPolicy
-import org.apache.nifi.registry.authorization.AuthorizableLookup
-import org.apache.nifi.registry.authorization.ConfigurableAccessPolicyProvider
-import org.apache.nifi.registry.authorization.ConfigurableUserGroupProvider
-import org.apache.nifi.registry.authorization.Group
-import org.apache.nifi.registry.authorization.RequestAction
-import org.apache.nifi.registry.authorization.StandardManagedAuthorizer
-import org.apache.nifi.registry.authorization.User as AuthUser
-import org.apache.nifi.registry.authorization.exception.AccessDeniedException
-import org.apache.nifi.registry.authorization.resource.Authorizable
-import org.apache.nifi.registry.authorization.resource.ResourceType
+import org.apache.nifi.registry.security.authorization.AccessPolicy as AuthAccessPolicy
+import org.apache.nifi.registry.security.authorization.AuthorizableLookup
+import org.apache.nifi.registry.security.authorization.ConfigurableAccessPolicyProvider
+import org.apache.nifi.registry.security.authorization.ConfigurableUserGroupProvider
+import org.apache.nifi.registry.security.authorization.Group
+import org.apache.nifi.registry.security.authorization.RequestAction
+import org.apache.nifi.registry.security.authorization.StandardManagedAuthorizer
+import org.apache.nifi.registry.security.authorization.User as AuthUser
+import org.apache.nifi.registry.security.authorization.exception.AccessDeniedException
+import org.apache.nifi.registry.security.authorization.resource.Authorizable
+import org.apache.nifi.registry.security.authorization.resource.ResourceType
 import org.apache.nifi.registry.bucket.Bucket
 import org.apache.nifi.registry.model.authorization.AccessPolicy
 import org.apache.nifi.registry.model.authorization.User

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestKeyRepository.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestKeyRepository.java b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestKeyRepository.java
new file mode 100644
index 0000000..567b0be
--- /dev/null
+++ b/nifi-registry-framework/src/test/java/org/apache/nifi/registry/db/repository/TestKeyRepository.java
@@ -0,0 +1,106 @@
+/*
+ * 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.db.repository;
+
+import org.apache.nifi.registry.db.DatabaseBaseTest;
+import org.apache.nifi.registry.db.entity.KeyEntity;
+import org.junit.Test;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.transaction.annotation.Transactional;
+
+import java.util.UUID;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+
+public class TestKeyRepository extends DatabaseBaseTest {
+
+    @Autowired
+    private KeyRepository keyRepository;
+
+    @Test
+    public void testCreate() {
+        final KeyEntity key = new KeyEntity();
+        key.setId(UUID.randomUUID().toString());
+        key.setTenantIdentity("user");
+        key.setKeyValue(UUID.randomUUID().toString());
+
+        final KeyEntity createdKey = keyRepository.save(key);
+        assertNotNull(createdKey);
+        assertEquals(key.getId(), createdKey.getId());
+        assertEquals(key.getTenantIdentity(), createdKey.getTenantIdentity());
+        assertEquals(key.getKeyValue(), createdKey.getKeyValue());
+    }
+
+    @Test
+    public void testUpdate() {
+        final String prepopulatedKeyId = "1";  // see test-setup.sql
+
+        final KeyEntity existingKey = keyRepository.findOne(prepopulatedKeyId);
+        assertNotNull(existingKey);
+
+        final String updatedKeyValue = existingKey.getKeyValue() + " UPDATED";
+        existingKey.setKeyValue(updatedKeyValue);
+
+        keyRepository.save(existingKey);
+
+        final KeyEntity updatedKey = keyRepository.findOne(prepopulatedKeyId);
+        assertNotNull(updatedKey);
+        assertEquals(updatedKeyValue, updatedKey.getKeyValue());
+    }
+
+    @Test
+    @Transactional
+    public void testDelete() {
+        final String id = "1";
+
+        final KeyEntity existingKey = keyRepository.findOne("1");
+        assertNotNull(existingKey);
+
+        keyRepository.delete(existingKey);
+
+        final KeyEntity deletedKey = keyRepository.findOne("1");
+        assertNull(deletedKey);
+    }
+
+    @Test
+    public void testFindOneByTenantIdentity() {
+        final String prepopulatedKeyTenantIdentity = "unit_test_tenant_identity";  // see test-setup.sql
+
+        final KeyEntity existingKey = keyRepository.findOneByTenantIdentity(prepopulatedKeyTenantIdentity);
+        assertNotNull(existingKey);
+        assertEquals("1", existingKey.getId());
+    }
+
+    @Test
+    @Transactional
+    public void testDeleteByTenantIdentity() {
+        final String prepopulatedKeyTenantIdentity = "unit_test_tenant_identity";  // see test-setup.sql
+
+        final KeyEntity existingKey = keyRepository.findOneByTenantIdentity(prepopulatedKeyTenantIdentity);
+        assertNotNull(existingKey);
+
+        keyRepository.delete(existingKey);
+
+        KeyEntity deletedKey = keyRepository.findOneByTenantIdentity(prepopulatedKeyTenantIdentity);
+        assertNull(deletedKey);
+        deletedKey = keyRepository.findOne("1");
+        assertNull(deletedKey);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/test/resources/db/migration/V999999.1__test-setup.sql
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/test/resources/db/migration/V999999.1__test-setup.sql b/nifi-registry-framework/src/test/resources/db/migration/V999999.1__test-setup.sql
index 26fe647..8d9d6fb 100644
--- a/nifi-registry-framework/src/test/resources/db/migration/V999999.1__test-setup.sql
+++ b/nifi-registry-framework/src/test/resources/db/migration/V999999.1__test-setup.sql
@@ -61,4 +61,10 @@ insert into flow_snapshot (flow_id, version, created, comments)
   values ('1', 2, parsedatetime('2017-09-11 12:58:00.000 UTC', 'yyyy-MM-dd hh:mm:ss.SSS z'), 'This is flow 1 snapshot 2');
 
 insert into flow_snapshot (flow_id, version, created, comments)
-  values ('1', 3, parsedatetime('2017-09-11 12:59:00.000 UTC', 'yyyy-MM-dd hh:mm:ss.SSS z'), 'This is flow 1 snapshot 3');
\ No newline at end of file
+  values ('1', 3, parsedatetime('2017-09-11 12:59:00.000 UTC', 'yyyy-MM-dd hh:mm:ss.SSS z'), 'This is flow 1 snapshot 3');
+
+
+-- test data for signing keys
+
+insert into signing_key (id, tenant_identity, key_value)
+  values ('1', 'unit_test_tenant_identity', '0123456789abcdef');
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/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 b40b784..b0090c0 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
@@ -49,6 +49,8 @@ public class NiFiRegistryProperties extends Properties {
     public static final String SECURITY_NEED_CLIENT_AUTH = "nifi.registry.security.needClientAuth";
     public static final String SECURITY_AUTHORIZERS_CONFIGURATION_FILE = "nifi.registry.security.authorizers.configuration.file";
     public static final String SECURITY_AUTHORIZER = "nifi.registry.security.authorizer";
+    public static final String SECURITY_IDENTITY_PROVIDERS_CONFIGURATION_FILE = "nifi.registry.security.identity.providers.configuration.file";
+    public static final String SECURITY_IDENTITY_PROVIDER = "nifi.registry.security.identity.provider";
     public static final String SECURITY_IDENTITY_MAPPING_PATTERN_PREFIX = "nifi.registry.security.identity.mapping.pattern.";
     public static final String SECURITY_IDENTITY_MAPPING_VALUE_PREFIX = "nifi.registry.security.identity.mapping.value.";
 
@@ -64,6 +66,7 @@ public class NiFiRegistryProperties extends Properties {
     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 int getWebThreads() {
         int webThreads = 200;
@@ -178,6 +181,15 @@ public class NiFiRegistryProperties extends Properties {
         }
     }
 
+    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);
+        }
+    }
+
     public Set<String> getExtensionsDirs() {
         final Set<String> extensionDirs = new HashSet<>();
         stringPropertyNames().stream().filter(key -> key.startsWith(EXTENSION_DIR_PREFIX)).forEach(key -> extensionDirs.add(getProperty(key)));

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-resources/src/main/resources/conf/authorizers.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-resources/src/main/resources/conf/authorizers.xml b/nifi-registry-resources/src/main/resources/conf/authorizers.xml
index e4696bb..c3cad80 100644
--- a/nifi-registry-resources/src/main/resources/conf/authorizers.xml
+++ b/nifi-registry-resources/src/main/resources/conf/authorizers.xml
@@ -42,12 +42,112 @@
     -->
     <userGroupProvider>
         <identifier>file-user-group-provider</identifier>
-        <class>org.apache.nifi.registry.authorization.file.FileUserGroupProvider</class>
+        <class>org.apache.nifi.registry.security.authorization.file.FileUserGroupProvider</class>
         <property name="Users File">./conf/users.xml</property>
         <property name="Initial User Identity 1"><!--CN=abc, OU=xyz--></property>
     </userGroupProvider>
 
     <!--
+        The LdapUserGroupProvider will retrieve users and groups from an LDAP server. The users and groups
+        are not configurable.
+
+        'Authentication Strategy' - How the connection to the LDAP server is authenticated. Possible
+            values are ANONYMOUS, SIMPLE, LDAPS, or START_TLS.
+
+        'Manager DN' - The DN of the manager that is used to bind to the LDAP server to search for users.
+        'Manager Password' - The password of the manager that is used to bind to the LDAP server to
+            search for users.
+
+        'TLS - Keystore' - Path to the Keystore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Keystore Password' - Password for the Keystore that is used when connecting to LDAP
+            using LDAPS or START_TLS.
+        'TLS - Keystore Type' - Type of the Keystore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Truststore' - Path to the Truststore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Password' - Password for the Truststore that is used when connecting to
+            LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Type' - Type of the Truststore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Client Auth' - Client authentication policy when connecting to LDAP using LDAPS or START_TLS.
+            Possible values are REQUIRED, WANT, NONE.
+        'TLS - Protocol' - Protocol to use when connecting to LDAP using LDAPS or START_TLS. (i.e. TLS,
+            TLSv1.1, TLSv1.2, etc).
+        'TLS - Shutdown Gracefully' - Specifies whether the TLS should be shut down gracefully
+            before the target context is closed. Defaults to false.
+
+        'Referral Strategy' - Strategy for handling referrals. Possible values are FOLLOW, IGNORE, THROW.
+        'Connect Timeout' - Duration of connect timeout. (i.e. 10 secs).
+        'Read Timeout' - Duration of read timeout. (i.e. 10 secs).
+
+        'Url' - Space-separated list of URLs of the LDAP servers (i.e. ldap://<hostname>:<port>).
+        'Page Size' - Sets the page size when retrieving users and groups. If not specified, no paging is performed.
+        'Sync Interval' - Duration of time between syncing users and groups. (i.e. 30 mins).
+
+        'User Search Base' - Base DN for searching for users (i.e. ou=users,o=nifi). Required to search users.
+        'User Object Class' - Object class for identifying users (i.e. person). Required if searching users.
+        'User Search Scope' - Search scope for searching users (ONE_LEVEL, OBJECT, or SUBTREE). Required if searching users.
+        'User Search Filter' - Filter for searching for users against the 'User Search Base' (i.e. (memberof=cn=team1,ou=groups,o=nifi) ). Optional.
+        'User Identity Attribute' - Attribute to use to extract user identity (i.e. cn). Optional. If not set, the entire DN is used.
+        'User Group Name Attribute' - Attribute to use to define group membership (i.e. memberof). Optional. If not set
+            group membership will not be calculated through the users. Will rely on group membership being defined
+            through 'Group Member Attribute' if set.
+
+        'Group Search Base' - Base DN for searching for groups (i.e. ou=groups,o=nifi). Required to search groups.
+        'Group Object Class' - Object class for identifying groups (i.e. groupOfNames). Required if searching groups.
+        'Group Search Scope' - Search scope for searching groups (ONE_LEVEL, OBJECT, or SUBTREE). Required if searching groups.
+        'Group Search Filter' - Filter for searching for groups against the 'Group Search Base'. Optional.
+        'Group Name Attribute' - Attribute to use to extract group name (i.e. cn). Optional. If not set, the entire DN is used.
+        'Group Member Attribute' - Attribute to use to define group membership (i.e. member). Optional. If not set
+            group membership will not be calculated through the groups. Will rely on group member being defined
+            through 'User Group Name Attribute' if set.
+
+        NOTE: Any identity mapping rules specified in nifi.properties will also be applied to the user identities.
+            Group names are not mapped.
+    -->
+    <!-- To enable the ldap-user-group-provider remove 2 lines. This is 1 of 2.
+    <userGroupProvider>
+        <identifier>ldap-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.ldap.tenants.LdapUserGroupProvider</class>
+        <property name="Authentication Strategy">START_TLS</property>
+
+        <property name="Manager DN"></property>
+        <property name="Manager Password"></property>
+
+        <property name="TLS - Keystore"></property>
+        <property name="TLS - Keystore Password"></property>
+        <property name="TLS - Keystore Type"></property>
+        <property name="TLS - Truststore"></property>
+        <property name="TLS - Truststore Password"></property>
+        <property name="TLS - Truststore Type"></property>
+        <property name="TLS - Client Auth"></property>
+        <property name="TLS - Protocol"></property>
+        <property name="TLS - Shutdown Gracefully"></property>
+
+        <property name="Referral Strategy">FOLLOW</property>
+        <property name="Connect Timeout">10 secs</property>
+        <property name="Read Timeout">10 secs</property>
+
+        <property name="Url"></property>
+        <property name="Page Size"></property>
+        <property name="Sync Interval">30 mins</property>
+
+        <property name="User Search Base"></property>
+        <property name="User Object Class">person</property>
+        <property name="User Search Scope">ONE_LEVEL</property>
+        <property name="User Search Filter"></property>
+        <property name="User Identity Attribute"></property>
+        <property name="User Group Name Attribute"></property>
+
+        <property name="Group Search Base"></property>
+        <property name="Group Object Class">group</property>
+        <property name="Group Search Scope">ONE_LEVEL</property>
+        <property name="Group Search Filter"></property>
+        <property name="Group Name Attribute"></property>
+        <property name="Group Member Attribute"></property>
+    </userGroupProvider>
+    To enable the ldap-user-group-provider remove 2 lines. This is 2 of 2. -->
+
+    <!--
         The CompositeUserGroupProvider will provide support for retrieving users and groups from multiple sources.
 
         - User Group Provider [unique key] - The identifier of user group providers to load from. The name of
@@ -116,7 +216,7 @@
     -->
     <accessPolicyProvider>
         <identifier>file-access-policy-provider</identifier>
-        <class>org.apache.nifi.registry.authorization.file.FileAccessPolicyProvider</class>
+        <class>org.apache.nifi.registry.security.authorization.file.FileAccessPolicyProvider</class>
         <property name="User Group Provider">file-user-group-provider</property>
         <property name="Authorizations File">./conf/authorizations.xml</property>
         <property name="Initial Admin Identity"><!-- CN=abc, OU=xyz --></property>
@@ -134,7 +234,7 @@
     -->
     <authorizer>
         <identifier>managed-authorizer</identifier>
-        <class>org.apache.nifi.registry.authorization.StandardManagedAuthorizer</class>
+        <class>org.apache.nifi.registry.security.authorization.StandardManagedAuthorizer</class>
         <property name="Access Policy Provider">file-access-policy-provider</property>
     </authorizer>
 

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/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
new file mode 100644
index 0000000..6fcaf7c
--- /dev/null
+++ b/nifi-registry-resources/src/main/resources/conf/identity-providers.xml
@@ -0,0 +1,90 @@
+<?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 identity providers to use when running securely. In order
+    to use a specific provider it must be configured here and its identifier
+    must be specified in the nifi-registry.properties file.
+-->
+<identityProviders>
+    <!--
+        Identity Provider for users logging in with username/password against an LDAP server.
+        
+        'Authentication Strategy' - How the connection to the LDAP server is authenticated. Possible
+            values are ANONYMOUS, SIMPLE, LDAPS, or START_TLS.
+        
+        'Manager DN' - The DN of the manager that is used to bind to the LDAP server to search for users.
+        'Manager Password' - The password of the manager that is used to bind to the LDAP server to
+            search for users.
+            
+        'TLS - Keystore' - Path to the Keystore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Keystore Password' - Password for the Keystore that is used when connecting to LDAP
+            using LDAPS or START_TLS.
+        'TLS - Keystore Type' - Type of the Keystore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Truststore' - Path to the Truststore that is used when connecting to LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Password' - Password for the Truststore that is used when connecting to
+            LDAP using LDAPS or START_TLS.
+        'TLS - Truststore Type' - Type of the Truststore that is used when connecting to LDAP using
+            LDAPS or START_TLS (i.e. JKS or PKCS12).
+        'TLS - Client Auth' - Client authentication policy when connecting to LDAP using LDAPS or START_TLS.
+            Possible values are REQUIRED, WANT, NONE.
+        'TLS - Protocol' - Protocol to use when connecting to LDAP using LDAPS or START_TLS. (i.e. TLS,
+            TLSv1.1, TLSv1.2, etc).
+        'TLS - Shutdown Gracefully' - Specifies whether the TLS should be shut down gracefully 
+            before the target context is closed. Defaults to false.
+            
+        'Referral Strategy' - Strategy for handling referrals. Possible values are FOLLOW, IGNORE, THROW.
+        'Connect Timeout' - Duration of connect timeout. (i.e. 10 secs).
+        'Read Timeout' - Duration of read timeout. (i.e. 10 secs).
+       
+        'Url' - Space-separated list of URLs of the LDAP servers (i.e. ldap://<hostname>:<port>).
+        'User Search Base' - Base DN for searching for users (i.e. CN=Users,DC=example,DC=com).
+        'User Search Filter' - Filter for searching for users against the 'User Search Base'.
+            (i.e. sAMAccountName={0}). The user specified name is inserted into '{0}'.
+
+        'Identity Strategy' - Strategy to identify users. Possible values are USE_DN and USE_USERNAME.
+            The default functionality if this property is missing is USE_DN in order to retain
+            backward compatibility. USE_DN will use the full DN of the user entry if possible.
+            USE_USERNAME will use the username the user logged in with.
+        '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 ldap-identity-provider remove 2 lines. This is 1 of 2.
+    <provider>
+        <identifier>ldap-identity-provider</identifier>
+        <class>org.apache.nifi.registry.security.ldap.LdapIdentityProvider</class>
+        <property name="Authentication Strategy">SIMPLE</property>
+
+        <property name="Manager DN"></property>
+        <property name="Manager Password"></property>
+        
+        <property name="Referral Strategy">FOLLOW</property>
+        <property name="Connect Timeout">10 secs</property>
+        <property name="Read Timeout">10 secs</property>
+
+        <property name="Url"></property>
+        <property name="User Search Base"></property>
+        <property name="User Search Filter"></property>
+
+        <property name="Identity Strategy">USE_USERNAME</property>
+        <property name="Authentication Expiration">12 hours</property>
+    </provider>
+    To enable the ldap-identity-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/90f36dd2/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 cd5fd8b..7f0c7ae 100644
--- a/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
+++ b/nifi-registry-resources/src/main/resources/conf/nifi-registry.properties
@@ -33,6 +33,8 @@ nifi.registry.security.truststorePasswd=${nifi.registry.security.truststorePassw
 nifi.registry.security.needClientAuth=${nifi.registry.security.needClientAuth}
 nifi.registry.security.authorizers.configuration.file=${nifi.registry.security.authorizers.configuration.file}
 nifi.registry.security.authorizer=${nifi.registry.security.authorizer}
+nifi.registry.security.identity.providers.configuration.file=${nifi.registry.security.identity.providers.configuration.file}
+nifi.registry.security.identity.provider=${nifi.registry.security.identity.provider}
 
 # providers properties #
 nifi.registry.providers.configuration.file=${nifi.registry.providers.configuration.file}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api-impl/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/pom.xml b/nifi-registry-security-api-impl/pom.xml
deleted file mode 100644
index efacb88..0000000
--- a/nifi-registry-security-api-impl/pom.xml
+++ /dev/null
@@ -1,100 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ 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.
-  -->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <parent>
-        <artifactId>nifi-registry</artifactId>
-        <groupId>org.apache.nifi.registry</groupId>
-        <version>0.0.1-SNAPSHOT</version>
-    </parent>
-    <modelVersion>4.0.0</modelVersion>
-
-    <artifactId>nifi-registry-security-api-impl</artifactId>
-    <packaging>jar</packaging>
-
-    <build>
-        <resources>
-            <resource>
-                <directory>src/main/xsd</directory>
-            </resource>
-            <resource>
-                <directory>src/main/resources</directory>
-            </resource>
-        </resources>
-        <plugins>
-            <plugin>
-                <groupId>org.codehaus.mojo</groupId>
-                <artifactId>jaxb2-maven-plugin</artifactId>
-                <executions>
-                    <execution>
-                        <id>authorizations</id>
-                        <goals>
-                            <goal>xjc</goal>
-                        </goals>
-                        <configuration>
-                            <sources>
-                                <source>src/main/xsd/authorizations.xsd</source>
-                            </sources>
-                            <packageName>org.apache.nifi.registry.authorization.file.generated</packageName>
-                        </configuration>
-                    </execution>
-                    <execution>
-                        <id>tenants</id>
-                        <goals>
-                            <goal>xjc</goal>
-                        </goals>
-                        <configuration>
-                            <sources>
-                                <source>src/main/xsd/tenants.xsd</source>
-                            </sources>
-                            <packageName>org.apache.nifi.registry.authorization.file.tenants.generated</packageName>
-                            <clearOutputDir>false</clearOutputDir>
-                        </configuration>
-                    </execution>
-                </executions>
-                <configuration>
-                    <outputDirectory>${project.build.directory}/generated-sources/jaxb</outputDirectory>
-                </configuration>
-            </plugin>
-            <plugin>
-                <groupId>org.apache.maven.plugins</groupId>
-                <artifactId>maven-checkstyle-plugin</artifactId>
-                <configuration>
-                    <excludes>**/authorization/file/generated/*.java,**/authorization/file/tenants/generated/*.java</excludes>
-                </configuration>
-            </plugin>
-        </plugins>
-    </build>
-
-    <dependencies>
-        <dependency>
-            <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-properties</artifactId>
-            <version>0.0.1-SNAPSHOT</version>
-            <scope>provided</scope>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-security-api</artifactId>
-            <version>0.0.1-SNAPSHOT</version>
-            <scope>provided</scope>
-        </dependency>
-    </dependencies>
-
-</project>
\ No newline at end of file


[14/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java
new file mode 100644
index 0000000..1ca6368
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerFactory.java
@@ -0,0 +1,809 @@
+/*
+ * 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.authorization;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.extension.ExtensionManager;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.provider.StandardProviderFactory;
+import org.apache.nifi.registry.security.authorization.annotation.AuthorizerContext;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.registry.security.authorization.exception.UninheritableAuthorizationsException;
+import org.apache.nifi.registry.security.authorization.generated.Authorizers;
+import org.apache.nifi.registry.security.authorization.generated.Prop;
+import org.apache.nifi.registry.security.util.XmlUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.DisposableBean;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.xml.sax.SAXException;
+
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import java.io.File;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * This implementation of AuthorizerFactory in NiFi Registry is based on a combination of
+ * NiFi's AuthorizerFactory and AuthorizerFactoryBean.
+ */
+@Configuration("authorizerFactory")
+public class AuthorizerFactory implements UserGroupProviderLookup, AccessPolicyProviderLookup, AuthorizerLookup, DisposableBean {
+
+    private static final Logger logger = LoggerFactory.getLogger(StandardProviderFactory.class);
+
+    private static final String AUTHORIZERS_XSD = "/authorizers.xsd";
+    private static final String JAXB_GENERATED_PATH = "org.apache.nifi.registry.security.authorization.generated";
+    private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
+
+    /**
+     * Load the JAXBContext.
+     */
+    private static JAXBContext initializeJaxbContext() {
+        try {
+            return JAXBContext.newInstance(JAXB_GENERATED_PATH, AuthorizerFactory.class.getClassLoader());
+        } catch (JAXBException e) {
+            throw new RuntimeException("Unable to create JAXBContext.", e);
+        }
+    }
+
+    private final NiFiRegistryProperties properties;
+    private final ExtensionManager extensionManager;
+
+    private Authorizer authorizer;
+    private final Map<String, UserGroupProvider> userGroupProviders = new HashMap<>();
+    private final Map<String, AccessPolicyProvider> accessPolicyProviders = new HashMap<>();
+    private final Map<String, Authorizer> authorizers = new HashMap<>();
+
+    @Autowired
+    public AuthorizerFactory(final NiFiRegistryProperties properties, final ExtensionManager extensionManager) {
+        this.properties = properties;
+        this.extensionManager = extensionManager;
+
+        if (this.properties == null) {
+            throw new IllegalStateException("NiFiRegistryProperties cannot be null");
+        }
+
+        if (this.extensionManager == null) {
+            throw new IllegalStateException("ExtensionManager cannot be null");
+        }
+    }
+
+    /***** UserGroupProviderLookup *****/
+
+    @Override
+    public UserGroupProvider getUserGroupProvider(String identifier) {
+        return userGroupProviders.get(identifier);
+    }
+
+    /***** AccessPolicyProviderLookup *****/
+
+    @Override
+    public AccessPolicyProvider getAccessPolicyProvider(String identifier) {
+        return accessPolicyProviders.get(identifier);
+    }
+
+
+    /***** AuthorizerLookup *****/
+
+    @Override
+    public Authorizer getAuthorizer(String identifier) {
+        return authorizers.get(identifier);
+    }
+
+    /***** AuthorizerFactory / DisposableBean *****/
+
+    @Bean
+    public Authorizer getAuthorizer() throws AuthorizerFactoryException {
+        if (authorizer == null) {
+            if (properties.getSslPort() == null) {
+                // use a default authorizer... only allowable when running not securely
+                authorizer = createDefaultAuthorizer();
+            } else {
+                // look up the authorizer to use
+                final String authorizerIdentifier = properties.getProperty(NiFiRegistryProperties.SECURITY_AUTHORIZER);
+
+                // ensure the authorizer class name was specified
+                if (StringUtils.isBlank(authorizerIdentifier)) {
+                    throw new AuthorizerFactoryException("When running securely, the authorizer identifier must be specified in the nifi properties file.");
+                } else {
+
+                    try {
+                        final Authorizers authorizerConfiguration = loadAuthorizersConfiguration();
+
+                        // create each user group provider
+                        for (final org.apache.nifi.registry.security.authorization.generated.UserGroupProvider userGroupProvider : authorizerConfiguration.getUserGroupProvider()) {
+                            userGroupProviders.put(userGroupProvider.getIdentifier(), createUserGroupProvider(userGroupProvider.getIdentifier(), userGroupProvider.getClazz()));
+                        }
+
+                        // configure each user group provider
+                        for (final org.apache.nifi.registry.security.authorization.generated.UserGroupProvider provider : authorizerConfiguration.getUserGroupProvider()) {
+                            final UserGroupProvider instance = userGroupProviders.get(provider.getIdentifier());
+                            instance.onConfigured(loadAuthorizerConfiguration(provider.getIdentifier(), provider.getProperty()));
+                        }
+
+                        // create each access policy provider
+                        for (final org.apache.nifi.registry.security.authorization.generated.AccessPolicyProvider accessPolicyProvider : authorizerConfiguration.getAccessPolicyProvider()) {
+                            accessPolicyProviders.put(accessPolicyProvider.getIdentifier(), createAccessPolicyProvider(accessPolicyProvider.getIdentifier(), accessPolicyProvider.getClazz()));
+                        }
+
+                        // configure each access policy provider
+                        for (final org.apache.nifi.registry.security.authorization.generated.AccessPolicyProvider provider : authorizerConfiguration.getAccessPolicyProvider()) {
+                            final AccessPolicyProvider instance = accessPolicyProviders.get(provider.getIdentifier());
+                            instance.onConfigured(loadAuthorizerConfiguration(provider.getIdentifier(), provider.getProperty()));
+                        }
+
+                        // create each authorizer
+                        for (final org.apache.nifi.registry.security.authorization.generated.Authorizer authorizer : authorizerConfiguration.getAuthorizer()) {
+                            authorizers.put(authorizer.getIdentifier(), createAuthorizer(authorizer.getIdentifier(), authorizer.getClazz(), authorizer.getClasspath()));
+                        }
+
+                        // configure each authorizer
+                        for (final org.apache.nifi.registry.security.authorization.generated.Authorizer provider : authorizerConfiguration.getAuthorizer()) {
+                            final Authorizer instance = authorizers.get(provider.getIdentifier());
+                            instance.onConfigured(loadAuthorizerConfiguration(provider.getIdentifier(), provider.getProperty()));
+                        }
+
+                        // get the authorizer instance
+                        authorizer = getAuthorizer(authorizerIdentifier);
+
+                        // ensure it was found
+                        if (authorizer == null) {
+                            throw new AuthorizerFactoryException(String.format("The specified authorizer '%s' could not be found.", authorizerIdentifier));
+                        }
+                    } catch (Exception e) {
+                        throw new AuthorizerFactoryException("Failed to construct Authorizer.", e);
+                    }
+                }
+            }
+        }
+        return authorizer;
+    }
+
+    @Override
+    public void destroy() throws Exception {
+        if (authorizers != null) {
+            authorizers.entrySet().stream().forEach(e -> e.getValue().preDestruction());
+        }
+
+        if (accessPolicyProviders != null) {
+            accessPolicyProviders.entrySet().stream().forEach(e -> e.getValue().preDestruction());
+        }
+
+        if (userGroupProviders != null) {
+            userGroupProviders.entrySet().stream().forEach(e -> e.getValue().preDestruction());
+        }
+    }
+
+    private Authorizers loadAuthorizersConfiguration() throws Exception {
+        final File authorizersConfigurationFile = properties.getAuthorizersConfigurationFile();
+
+        // load the authorizers from the specified file
+        if (authorizersConfigurationFile.exists()) {
+            try {
+                // find the schema
+                final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
+                final Schema schema = schemaFactory.newSchema(Authorizers.class.getResource(AUTHORIZERS_XSD));
+
+                // attempt to unmarshal
+                final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
+                unmarshaller.setSchema(schema);
+                final JAXBElement<Authorizers> element = unmarshaller.unmarshal(XmlUtils.createSafeReader(new StreamSource(authorizersConfigurationFile)), Authorizers.class);
+                return element.getValue();
+            } catch (SAXException | JAXBException e) {
+                throw new Exception("Unable to load the authorizer configuration file at: " + authorizersConfigurationFile.getAbsolutePath(), e);
+            }
+        } else {
+            throw new Exception("Unable to find the authorizer configuration file at " + authorizersConfigurationFile.getAbsolutePath());
+        }
+    }
+
+    private AuthorizerConfigurationContext loadAuthorizerConfiguration(final String identifier, final List<Prop> properties) {
+        final Map<String, String> authorizerProperties = new HashMap<>();
+
+        for (final Prop property : properties) {
+            authorizerProperties.put(property.getName(), property.getValue());
+        }
+        return new StandardAuthorizerConfigurationContext(identifier, authorizerProperties);
+    }
+
+    private UserGroupProvider createUserGroupProvider(final String identifier, final String userGroupProviderClassName) throws Exception {
+
+        final UserGroupProvider instance;
+
+        final ClassLoader classLoader = extensionManager.getExtensionClassLoader(userGroupProviderClassName);
+        if (classLoader == null) {
+            throw new IllegalStateException("Extension not found in any of the configured class loaders: " + userGroupProviderClassName);
+        }
+
+        // attempt to load the class
+        Class<?> rawUserGroupProviderClass = Class.forName(userGroupProviderClassName, true, classLoader);
+        Class<? extends UserGroupProvider> userGroupProviderClass = rawUserGroupProviderClass.asSubclass(UserGroupProvider.class);
+
+        // otherwise create a new instance
+        Constructor constructor = userGroupProviderClass.getConstructor();
+        instance = (UserGroupProvider) constructor.newInstance();
+
+        // method injection
+        performMethodInjection(instance, userGroupProviderClass);
+
+        // field injection
+        performFieldInjection(instance, userGroupProviderClass);
+
+        // call post construction lifecycle event
+        instance.initialize(new StandardAuthorizerInitializationContext(identifier, this, this, this));
+
+        return instance;
+    }
+
+    private AccessPolicyProvider createAccessPolicyProvider(final String identifier, final String accessPolicyProviderClassName) throws Exception {
+        final AccessPolicyProvider instance;
+
+        final ClassLoader classLoader = extensionManager.getExtensionClassLoader(accessPolicyProviderClassName);
+        if (classLoader == null) {
+            throw new IllegalStateException("Extension not found in any of the configured class loaders: " + accessPolicyProviderClassName);
+        }
+
+        // attempt to load the class
+        Class<?> rawAccessPolicyProviderClass = Class.forName(accessPolicyProviderClassName, true, classLoader);
+        Class<? extends AccessPolicyProvider> accessPolicyClass = rawAccessPolicyProviderClass.asSubclass(AccessPolicyProvider.class);
+
+        // otherwise create a new instance
+        Constructor constructor = accessPolicyClass.getConstructor();
+        instance = (AccessPolicyProvider) constructor.newInstance();
+
+        // method injection
+        performMethodInjection(instance, accessPolicyClass);
+
+        // field injection
+        performFieldInjection(instance, accessPolicyClass);
+
+        // call post construction lifecycle event
+        instance.initialize(new StandardAuthorizerInitializationContext(identifier, this, this, this));
+
+        return instance;
+    }
+
+    private Authorizer createAuthorizer(final String identifier, final String authorizerClassName, final String classpathResources) throws Exception {
+        final Authorizer instance;
+
+        final ClassLoader classLoader = extensionManager.getExtensionClassLoader(authorizerClassName);
+        if (classLoader == null) {
+            throw new IllegalStateException("Extension not found in any of the configured class loaders: " + authorizerClassName);
+        }
+
+        // attempt to load the class
+        Class<?> rawAuthorizerClass = Class.forName(authorizerClassName, true, classLoader);
+        Class<? extends Authorizer> authorizerClass = rawAuthorizerClass.asSubclass(Authorizer.class);
+
+        // otherwise create a new instance
+        Constructor constructor = authorizerClass.getConstructor();
+        instance = (Authorizer) constructor.newInstance();
+
+        // method injection
+        performMethodInjection(instance, authorizerClass);
+
+        // field injection
+        performFieldInjection(instance, authorizerClass);
+
+        // call post construction lifecycle event
+        instance.initialize(new StandardAuthorizerInitializationContext(identifier, this, this, this));
+
+        return installIntegrityChecks(instance);
+    }
+
+        private void performMethodInjection(final Object instance, final Class authorizerClass) throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
+        for (final Method method : authorizerClass.getMethods()) {
+            if (method.isAnnotationPresent(AuthorizerContext.class)) {
+                // make the method accessible
+                final boolean isAccessible = method.isAccessible();
+                method.setAccessible(true);
+
+                try {
+                    final Class<?>[] argumentTypes = method.getParameterTypes();
+
+                    // look for setters (single argument)
+                    if (argumentTypes.length == 1) {
+                        final Class<?> argumentType = argumentTypes[0];
+
+                        // look for well known types
+                        if (NiFiRegistryProperties.class.isAssignableFrom(argumentType)) {
+                            // nifi properties injection
+                            method.invoke(instance, properties);
+                        }
+                    }
+                } finally {
+                    method.setAccessible(isAccessible);
+                }
+            }
+        }
+
+        final Class parentClass = authorizerClass.getSuperclass();
+        if (parentClass != null && Authorizer.class.isAssignableFrom(parentClass)) {
+            performMethodInjection(instance, parentClass);
+        }
+    }
+
+    private void performFieldInjection(final Object instance, final Class authorizerClass) throws IllegalArgumentException, IllegalAccessException {
+        for (final Field field : authorizerClass.getDeclaredFields()) {
+            if (field.isAnnotationPresent(AuthorizerContext.class)) {
+                // make the method accessible
+                final boolean isAccessible = field.isAccessible();
+                field.setAccessible(true);
+
+                try {
+                    // get the type
+                    final Class<?> fieldType = field.getType();
+
+                    // only consider this field if it isn't set yet
+                    if (field.get(instance) == null) {
+                        // look for well known types
+                        if (NiFiRegistryProperties.class.isAssignableFrom(fieldType)) {
+                            // nifi properties injection
+                            field.set(instance, properties);
+                        }
+                    }
+
+                } finally {
+                    field.setAccessible(isAccessible);
+                }
+            }
+        }
+
+        final Class parentClass = authorizerClass.getSuperclass();
+        if (parentClass != null && Authorizer.class.isAssignableFrom(parentClass)) {
+            performFieldInjection(instance, parentClass);
+        }
+    }
+
+
+    /**
+     * @return a default Authorizer to use when running unsecurely with no authorizer configured
+     */
+    private Authorizer createDefaultAuthorizer() {
+        return new Authorizer() {
+            @Override
+            public AuthorizationResult authorize(final AuthorizationRequest request) throws AuthorizationAccessException {
+                return AuthorizationResult.approved();
+            }
+
+            @Override
+            public void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
+            }
+
+            @Override
+            public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+            }
+
+            @Override
+            public void preDestruction() throws AuthorizerDestructionException {
+            }
+        };
+    }
+
+    private static Authorizer installIntegrityChecks(final Authorizer baseAuthorizer) {
+        if (baseAuthorizer instanceof ManagedAuthorizer) {
+            final ManagedAuthorizer baseManagedAuthorizer = (ManagedAuthorizer) baseAuthorizer;
+            return new ManagedAuthorizer() {
+                @Override
+                public String getFingerprint() throws AuthorizationAccessException {
+                    return baseManagedAuthorizer.getFingerprint();
+                }
+
+                @Override
+                public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
+                    baseManagedAuthorizer.inheritFingerprint(fingerprint);
+                }
+
+                @Override
+                public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
+                    baseManagedAuthorizer.checkInheritability(proposedFingerprint);
+                }
+
+                @Override
+                public AccessPolicyProvider getAccessPolicyProvider() {
+                    final AccessPolicyProvider baseAccessPolicyProvider = baseManagedAuthorizer.getAccessPolicyProvider();
+                    if (baseAccessPolicyProvider instanceof ConfigurableAccessPolicyProvider) {
+                        final ConfigurableAccessPolicyProvider baseConfigurableAccessPolicyProvider = (ConfigurableAccessPolicyProvider) baseAccessPolicyProvider;
+                        return new ConfigurableAccessPolicyProvider() {
+                            @Override
+                            public String getFingerprint() throws AuthorizationAccessException {
+                                return baseConfigurableAccessPolicyProvider.getFingerprint();
+                            }
+
+                            @Override
+                            public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
+                                baseConfigurableAccessPolicyProvider.inheritFingerprint(fingerprint);
+                            }
+
+                            @Override
+                            public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
+                                baseConfigurableAccessPolicyProvider.checkInheritability(proposedFingerprint);
+                            }
+
+                            @Override
+                            public AccessPolicy addAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
+                                if (policyExists(baseConfigurableAccessPolicyProvider, accessPolicy)) {
+                                    throw new IllegalStateException(String.format("Found multiple policies for '%s' with '%s'.", accessPolicy.getResource(), accessPolicy.getAction()));
+                                }
+                                return baseConfigurableAccessPolicyProvider.addAccessPolicy(accessPolicy);
+                            }
+
+                            @Override
+                            public boolean isConfigurable(AccessPolicy accessPolicy) {
+                                return baseConfigurableAccessPolicyProvider.isConfigurable(accessPolicy);
+                            }
+
+                            @Override
+                            public AccessPolicy updateAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
+                                if (!baseConfigurableAccessPolicyProvider.isConfigurable(accessPolicy)) {
+                                    throw new IllegalArgumentException("The specified access policy is not support modification.");
+                                }
+                                return baseConfigurableAccessPolicyProvider.updateAccessPolicy(accessPolicy);
+                            }
+
+                            @Override
+                            public AccessPolicy deleteAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
+                                if (!baseConfigurableAccessPolicyProvider.isConfigurable(accessPolicy)) {
+                                    throw new IllegalArgumentException("The specified access policy is not support modification.");
+                                }
+                                return baseConfigurableAccessPolicyProvider.deleteAccessPolicy(accessPolicy);
+                            }
+
+                            @Override
+                            public AccessPolicy deleteAccessPolicy(String accessPolicyIdentifier) throws AuthorizationAccessException {
+                                if (!baseConfigurableAccessPolicyProvider.isConfigurable(baseConfigurableAccessPolicyProvider.getAccessPolicy(accessPolicyIdentifier))) {
+                                    throw new IllegalArgumentException("The specified access policy is not support modification.");
+                                }
+                                return baseConfigurableAccessPolicyProvider.deleteAccessPolicy(accessPolicyIdentifier);
+                            }
+
+                            @Override
+                            public Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException {
+                                return baseConfigurableAccessPolicyProvider.getAccessPolicies();
+                            }
+
+                            @Override
+                            public AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException {
+                                return baseConfigurableAccessPolicyProvider.getAccessPolicy(identifier);
+                            }
+
+                            @Override
+                            public AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) throws AuthorizationAccessException {
+                                return baseConfigurableAccessPolicyProvider.getAccessPolicy(resourceIdentifier, action);
+                            }
+
+                            @Override
+                            public UserGroupProvider getUserGroupProvider() {
+                                final UserGroupProvider baseUserGroupProvider = baseConfigurableAccessPolicyProvider.getUserGroupProvider();
+                                if (baseUserGroupProvider instanceof ConfigurableUserGroupProvider) {
+                                    final ConfigurableUserGroupProvider baseConfigurableUserGroupProvider = (ConfigurableUserGroupProvider) baseUserGroupProvider;
+                                    return new ConfigurableUserGroupProvider() {
+                                        @Override
+                                        public String getFingerprint() throws AuthorizationAccessException {
+                                            return baseConfigurableUserGroupProvider.getFingerprint();
+                                        }
+
+                                        @Override
+                                        public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
+                                            baseConfigurableUserGroupProvider.inheritFingerprint(fingerprint);
+                                        }
+
+                                        @Override
+                                        public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
+                                            baseConfigurableUserGroupProvider.checkInheritability(proposedFingerprint);
+                                        }
+
+                                        @Override
+                                        public User addUser(User user) throws AuthorizationAccessException {
+                                            if (tenantExists(baseConfigurableUserGroupProvider, user.getIdentifier(), user.getIdentity())) {
+                                                throw new IllegalStateException(String.format("User/user group already exists with the identity '%s'.", user.getIdentity()));
+                                            }
+                                            return baseConfigurableUserGroupProvider.addUser(user);
+                                        }
+
+                                        @Override
+                                        public boolean isConfigurable(User user) {
+                                            return baseConfigurableUserGroupProvider.isConfigurable(user);
+                                        }
+
+                                        @Override
+                                        public User updateUser(User user) throws AuthorizationAccessException {
+                                            if (tenantExists(baseConfigurableUserGroupProvider, user.getIdentifier(), user.getIdentity())) {
+                                                throw new IllegalStateException(String.format("User/user group already exists with the identity '%s'.", user.getIdentity()));
+                                            }
+                                            if (!baseConfigurableUserGroupProvider.isConfigurable(user)) {
+                                                throw new IllegalArgumentException("The specified user does not support modification.");
+                                            }
+                                            return baseConfigurableUserGroupProvider.updateUser(user);
+                                        }
+
+                                        @Override
+                                        public User deleteUser(User user) throws AuthorizationAccessException {
+                                            if (!baseConfigurableUserGroupProvider.isConfigurable(user)) {
+                                                throw new IllegalArgumentException("The specified user does not support modification.");
+                                            }
+                                            return baseConfigurableUserGroupProvider.deleteUser(user);
+                                        }
+
+                                        @Override
+                                        public User deleteUser(String userIdentifier) throws AuthorizationAccessException {
+                                            if (!baseConfigurableUserGroupProvider.isConfigurable(baseConfigurableUserGroupProvider.getUser(userIdentifier))) {
+                                                throw new IllegalArgumentException("The specified user does not support modification.");
+                                            }
+                                            return baseConfigurableUserGroupProvider.deleteUser(userIdentifier);
+                                        }
+
+                                        @Override
+                                        public Group addGroup(Group group) throws AuthorizationAccessException {
+                                            if (tenantExists(baseConfigurableUserGroupProvider, group.getIdentifier(), group.getName())) {
+                                                throw new IllegalStateException(String.format("User/user group already exists with the identity '%s'.", group.getName()));
+                                            }
+                                            return baseConfigurableUserGroupProvider.addGroup(group);
+                                        }
+
+                                        @Override
+                                        public boolean isConfigurable(Group group) {
+                                            return baseConfigurableUserGroupProvider.isConfigurable(group);
+                                        }
+
+                                        @Override
+                                        public Group updateGroup(Group group) throws AuthorizationAccessException {
+                                            if (tenantExists(baseConfigurableUserGroupProvider, group.getIdentifier(), group.getName())) {
+                                                throw new IllegalStateException(String.format("User/user group already exists with the identity '%s'.", group.getName()));
+                                            }
+                                            if (!baseConfigurableUserGroupProvider.isConfigurable(group)) {
+                                                throw new IllegalArgumentException("The specified group does not support modification.");
+                                            }
+                                            return baseConfigurableUserGroupProvider.updateGroup(group);
+                                        }
+
+                                        @Override
+                                        public Group deleteGroup(Group group) throws AuthorizationAccessException {
+                                            if (!baseConfigurableUserGroupProvider.isConfigurable(group)) {
+                                                throw new IllegalArgumentException("The specified group does not support modification.");
+                                            }
+                                            return baseConfigurableUserGroupProvider.deleteGroup(group);
+                                        }
+
+                                        @Override
+                                        public Group deleteGroup(String groupId) throws AuthorizationAccessException {
+                                            if (!baseConfigurableUserGroupProvider.isConfigurable(baseConfigurableUserGroupProvider.getGroup(groupId))) {
+                                                throw new IllegalArgumentException("The specified group does not support modification.");
+                                            }
+                                            return baseConfigurableUserGroupProvider.deleteGroup(groupId);
+                                        }
+
+                                        @Override
+                                        public Set<User> getUsers() throws AuthorizationAccessException {
+                                            return baseConfigurableUserGroupProvider.getUsers();
+                                        }
+
+                                        @Override
+                                        public User getUser(String identifier) throws AuthorizationAccessException {
+                                            return baseConfigurableUserGroupProvider.getUser(identifier);
+                                        }
+
+                                        @Override
+                                        public User getUserByIdentity(String identity) throws AuthorizationAccessException {
+                                            return baseConfigurableUserGroupProvider.getUserByIdentity(identity);
+                                        }
+
+                                        @Override
+                                        public Set<Group> getGroups() throws AuthorizationAccessException {
+                                            return baseConfigurableUserGroupProvider.getGroups();
+                                        }
+
+                                        @Override
+                                        public Group getGroup(String identifier) throws AuthorizationAccessException {
+                                            return baseConfigurableUserGroupProvider.getGroup(identifier);
+                                        }
+
+                                        @Override
+                                        public UserAndGroups getUserAndGroups(String identity) throws AuthorizationAccessException {
+                                            return baseConfigurableUserGroupProvider.getUserAndGroups(identity);
+                                        }
+
+                                        @Override
+                                        public void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
+                                            baseConfigurableUserGroupProvider.initialize(initializationContext);
+                                        }
+
+                                        @Override
+                                        public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+                                            baseConfigurableUserGroupProvider.onConfigured(configurationContext);
+                                        }
+
+                                        @Override
+                                        public void preDestruction() throws AuthorizerDestructionException {
+                                            baseConfigurableUserGroupProvider.preDestruction();
+                                        }
+                                    };
+                                } else {
+                                    return baseUserGroupProvider;
+                                }
+                            }
+
+                            @Override
+                            public void initialize(AccessPolicyProviderInitializationContext initializationContext) throws AuthorizerCreationException {
+                                baseConfigurableAccessPolicyProvider.initialize(initializationContext);
+                            }
+
+                            @Override
+                            public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+                                baseConfigurableAccessPolicyProvider.onConfigured(configurationContext);
+                            }
+
+                            @Override
+                            public void preDestruction() throws AuthorizerDestructionException {
+                                baseConfigurableAccessPolicyProvider.preDestruction();
+                            }
+                        };
+                    } else {
+                        return baseAccessPolicyProvider;
+                    }
+                }
+
+                @Override
+                public AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException {
+                    final AuthorizationResult result = baseAuthorizer.authorize(request);
+
+                    // audit the authorization request
+                    audit(baseAuthorizer, request, result);
+
+                    return result;
+                }
+
+                @Override
+                public void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
+                    baseManagedAuthorizer.initialize(initializationContext);
+                }
+
+                @Override
+                public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+                    baseManagedAuthorizer.onConfigured(configurationContext);
+
+                    final AccessPolicyProvider accessPolicyProvider = baseManagedAuthorizer.getAccessPolicyProvider();
+                    final UserGroupProvider userGroupProvider = accessPolicyProvider.getUserGroupProvider();
+
+                    // ensure that only one policy per resource-action exists
+                    for (AccessPolicy accessPolicy : accessPolicyProvider.getAccessPolicies()) {
+                        if (policyExists(accessPolicyProvider, accessPolicy)) {
+                            throw new AuthorizerCreationException(String.format("Found multiple policies for '%s' with '%s'.", accessPolicy.getResource(), accessPolicy.getAction()));
+                        }
+                    }
+
+                    // ensure that only one group exists per identity
+                    for (User user : userGroupProvider.getUsers()) {
+                        if (tenantExists(userGroupProvider, user.getIdentifier(), user.getIdentity())) {
+                            throw new AuthorizerCreationException(String.format("Found multiple users/user groups with identity '%s'.", user.getIdentity()));
+                        }
+                    }
+
+                    // ensure that only one group exists per identity
+                    for (Group group : userGroupProvider.getGroups()) {
+                        if (tenantExists(userGroupProvider, group.getIdentifier(), group.getName())) {
+                            throw new AuthorizerCreationException(String.format("Found multiple users/user groups with name '%s'.", group.getName()));
+                        }
+                    }
+                }
+
+                @Override
+                public void preDestruction() throws AuthorizerDestructionException {
+                    baseManagedAuthorizer.preDestruction();
+                }
+            };
+        } else {
+            return new Authorizer() {
+                @Override
+                public AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException {
+                    final AuthorizationResult result = baseAuthorizer.authorize(request);
+
+                    // audit the authorization request
+                    audit(baseAuthorizer, request, result);
+
+                    return result;
+                }
+
+                @Override
+                public void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
+                    baseAuthorizer.initialize(initializationContext);
+                }
+
+                @Override
+                public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+                    baseAuthorizer.onConfigured(configurationContext);
+                }
+
+                @Override
+                public void preDestruction() throws AuthorizerDestructionException {
+                    baseAuthorizer.preDestruction();
+                }
+            };
+        }
+    }
+
+    private static void audit(final Authorizer authorizer, final AuthorizationRequest request, final AuthorizationResult result) {
+        // audit when...
+        // 1 - the authorizer supports auditing
+        // 2 - the request is an access attempt
+        // 3 - the result is either approved/denied, when resource is not found a subsequent request may be following with the parent resource
+        if (authorizer instanceof AuthorizationAuditor && request.isAccessAttempt() && !AuthorizationResult.Result.ResourceNotFound.equals(result.getResult())) {
+            ((AuthorizationAuditor) authorizer).auditAccessAttempt(request, result);
+        }
+    }
+
+    /**
+     * Checks if another policy exists with the same resource and action as the given policy.
+     *
+     * @param checkAccessPolicy an access policy being checked
+     * @return true if another access policy exists with the same resource and action, false otherwise
+     */
+    private static boolean policyExists(final AccessPolicyProvider accessPolicyProvider, final AccessPolicy checkAccessPolicy) {
+        for (AccessPolicy accessPolicy : accessPolicyProvider.getAccessPolicies()) {
+            if (!accessPolicy.getIdentifier().equals(checkAccessPolicy.getIdentifier())
+                    && accessPolicy.getResource().equals(checkAccessPolicy.getResource())
+                    && accessPolicy.getAction().equals(checkAccessPolicy.getAction())) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Checks if another user exists with the same identity.
+     *
+     * @param identifier identity of the user
+     * @param identity identity of the user
+     * @return true if another user exists with the same identity, false otherwise
+     */
+    private static boolean tenantExists(final UserGroupProvider userGroupProvider, final String identifier, final String identity) {
+        for (User user : userGroupProvider.getUsers()) {
+            if (!user.getIdentifier().equals(identifier)
+                    && user.getIdentity().equals(identity)) {
+                return true;
+            }
+        }
+
+        for (Group group : userGroupProvider.getGroups()) {
+            if (!group.getIdentifier().equals(identifier)
+                    && group.getName().equals(identity)) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/CompositeConfigurableUserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/CompositeConfigurableUserGroupProvider.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/CompositeConfigurableUserGroupProvider.java
new file mode 100644
index 0000000..dc687ef
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/CompositeConfigurableUserGroupProvider.java
@@ -0,0 +1,205 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.registry.security.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.registry.security.authorization.ConfigurableUserGroupProvider;
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.User;
+import org.apache.nifi.registry.security.authorization.UserAndGroups;
+import org.apache.nifi.registry.security.authorization.UserGroupProvider;
+import org.apache.nifi.registry.security.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.registry.security.authorization.UserGroupProviderLookup;
+import org.apache.nifi.registry.security.authorization.exception.UninheritableAuthorizationsException;
+import org.apache.nifi.registry.util.PropertyValue;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class CompositeConfigurableUserGroupProvider extends CompositeUserGroupProvider implements ConfigurableUserGroupProvider {
+
+    static final String PROP_CONFIGURABLE_USER_GROUP_PROVIDER = "Configurable User Group Provider";
+
+    private UserGroupProviderLookup userGroupProviderLookup;
+    private ConfigurableUserGroupProvider configurableUserGroupProvider;
+
+    public CompositeConfigurableUserGroupProvider() {
+        super(true);
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
+        userGroupProviderLookup = initializationContext.getUserGroupProviderLookup();
+
+        // initialize the CompositeUserGroupProvider
+        super.initialize(initializationContext);
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        final PropertyValue configurableUserGroupProviderKey = configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER);
+        if (!configurableUserGroupProviderKey.isSet()) {
+            throw new AuthorizerCreationException("The Configurable User Group Provider must be set.");
+        }
+
+        final UserGroupProvider userGroupProvider = userGroupProviderLookup.getUserGroupProvider(configurableUserGroupProviderKey.getValue());
+
+        if (userGroupProvider == null) {
+            throw new AuthorizerCreationException(String.format("Unable to locate the Configurable User Group Provider: %s", configurableUserGroupProviderKey));
+        }
+
+        if (!(userGroupProvider instanceof ConfigurableUserGroupProvider)) {
+            throw new AuthorizerCreationException(String.format("The Configurable User Group Provider is not configurable: %s", configurableUserGroupProviderKey));
+        }
+
+        configurableUserGroupProvider = (ConfigurableUserGroupProvider) userGroupProvider;
+
+        // configure the CompositeUserGroupProvider
+        super.onConfigured(configurationContext);
+    }
+
+    @Override
+    public String getFingerprint() throws AuthorizationAccessException {
+        return configurableUserGroupProvider.getFingerprint();
+    }
+
+    @Override
+    public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
+        configurableUserGroupProvider.inheritFingerprint(fingerprint);
+    }
+
+    @Override
+    public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
+        configurableUserGroupProvider.checkInheritability(proposedFingerprint);
+    }
+
+    @Override
+    public User addUser(User user) throws AuthorizationAccessException {
+        return configurableUserGroupProvider.addUser(user);
+    }
+
+    @Override
+    public boolean isConfigurable(User user) {
+        return configurableUserGroupProvider.isConfigurable(user);
+    }
+
+    @Override
+    public User updateUser(User user) throws AuthorizationAccessException {
+        return configurableUserGroupProvider.updateUser(user);
+    }
+
+    @Override
+    public User deleteUser(User user) throws AuthorizationAccessException {
+        return configurableUserGroupProvider.deleteUser(user);
+    }
+
+    @Override
+    public User deleteUser(String userIdentifier) throws AuthorizationAccessException {
+        return configurableUserGroupProvider.deleteUser(userIdentifier);
+    }
+
+    @Override
+    public Group addGroup(Group group) throws AuthorizationAccessException {
+        return configurableUserGroupProvider.addGroup(group);
+    }
+
+    @Override
+    public boolean isConfigurable(Group group) {
+        return configurableUserGroupProvider.isConfigurable(group);
+    }
+
+    @Override
+    public Group updateGroup(Group group) throws AuthorizationAccessException {
+        return configurableUserGroupProvider.updateGroup(group);
+    }
+
+    @Override
+    public Group deleteGroup(Group group) throws AuthorizationAccessException {
+        return configurableUserGroupProvider.deleteGroup(group);
+    }
+
+    @Override
+    public Group deleteGroup(String groupIdentifier) throws AuthorizationAccessException {
+        return configurableUserGroupProvider.deleteGroup(groupIdentifier);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        final Set<User> users = new HashSet<>(configurableUserGroupProvider.getUsers());
+        users.addAll(super.getUsers());
+        return users;
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        User user = configurableUserGroupProvider.getUser(identifier);
+
+        if (user == null) {
+            user = super.getUser(identifier);
+        }
+
+        return user;
+    }
+
+    @Override
+    public User getUserByIdentity(String identity) throws AuthorizationAccessException {
+        User user = configurableUserGroupProvider.getUserByIdentity(identity);
+
+        if (user == null) {
+            user = super.getUserByIdentity(identity);
+        }
+
+        return user;
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        final Set<Group> groups = new HashSet<>(configurableUserGroupProvider.getGroups());
+        groups.addAll(super.getGroups());
+        return groups;
+    }
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        Group group = configurableUserGroupProvider.getGroup(identifier);
+
+        if (group == null) {
+            group = super.getGroup(identifier);
+        }
+
+        return group;
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String identity) throws AuthorizationAccessException {
+        UserAndGroups userAndGroups = configurableUserGroupProvider.getUserAndGroups(identity);
+
+        if (userAndGroups.getUser() == null) {
+            userAndGroups = super.getUserAndGroups(identity);
+        }
+
+        return userAndGroups;
+    }
+
+    @Override
+    public void preDestruction() throws AuthorizerDestructionException {
+        super.preDestruction();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/CompositeUserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/CompositeUserGroupProvider.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/CompositeUserGroupProvider.java
new file mode 100644
index 0000000..96e9d41
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/CompositeUserGroupProvider.java
@@ -0,0 +1,184 @@
+/*
+ * 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.authorization;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.security.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.User;
+import org.apache.nifi.registry.security.authorization.UserAndGroups;
+import org.apache.nifi.registry.security.authorization.UserGroupProvider;
+import org.apache.nifi.registry.security.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.registry.security.authorization.UserGroupProviderLookup;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class CompositeUserGroupProvider implements UserGroupProvider {
+
+    static final String PROP_USER_GROUP_PROVIDER_PREFIX = "User Group Provider ";
+    static final Pattern USER_GROUP_PROVIDER_PATTERN = Pattern.compile(PROP_USER_GROUP_PROVIDER_PREFIX + "\\S+");
+
+    private final boolean allowEmptyProviderList;
+
+    private UserGroupProviderLookup userGroupProviderLookup;
+    private List<UserGroupProvider> userGroupProviders = new ArrayList<>(); // order matters
+
+    public CompositeUserGroupProvider() {
+        this(false);
+    }
+
+    public CompositeUserGroupProvider(boolean allowEmptyProviderList) {
+        this.allowEmptyProviderList = allowEmptyProviderList;
+    }
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
+        userGroupProviderLookup = initializationContext.getUserGroupProviderLookup();
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        for (Map.Entry<String,String> entry : configurationContext.getProperties().entrySet()) {
+            Matcher matcher = USER_GROUP_PROVIDER_PATTERN.matcher(entry.getKey());
+            if (matcher.matches() && !StringUtils.isBlank(entry.getValue())) {
+                final String userGroupProviderKey = entry.getValue();
+                final UserGroupProvider userGroupProvider = userGroupProviderLookup.getUserGroupProvider(userGroupProviderKey);
+
+                if (userGroupProvider == null) {
+                    throw new AuthorizerCreationException(String.format("Unable to locate the configured User Group Provider: %s", userGroupProviderKey));
+                }
+
+                userGroupProviders.add(userGroupProvider);
+            }
+        }
+
+        if (!allowEmptyProviderList && userGroupProviders.isEmpty()) {
+            throw new AuthorizerCreationException("At least one User Group Provider must be configured.");
+        }
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        final Set<User> users = new HashSet<>();
+
+        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
+            users.addAll(userGroupProvider.getUsers());
+        }
+
+        return users;
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        User user = null;
+
+        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
+            user = userGroupProvider.getUser(identifier);
+
+            if (user != null) {
+                break;
+            }
+        }
+
+        return user;
+    }
+
+    @Override
+    public User getUserByIdentity(String identity) throws AuthorizationAccessException {
+        User user = null;
+
+        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
+            user = userGroupProvider.getUserByIdentity(identity);
+
+            if (user != null) {
+                break;
+            }
+        }
+
+        return user;
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        final Set<Group> groups = new HashSet<>();
+
+        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
+            groups.addAll(userGroupProvider.getGroups());
+        }
+
+        return groups;
+    }
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        Group group = null;
+
+        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
+            group = userGroupProvider.getGroup(identifier);
+
+            if (group != null) {
+                break;
+            }
+        }
+
+        return group;
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String identity) throws AuthorizationAccessException {
+        UserAndGroups userAndGroups = null;
+
+        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
+            userAndGroups = userGroupProvider.getUserAndGroups(identity);
+
+            if (userAndGroups.getUser() != null) {
+                break;
+            }
+        }
+
+        if (userAndGroups == null) {
+            // per API, returning non null with null user/groups
+            return new UserAndGroups() {
+                @Override
+                public User getUser() {
+                    return null;
+                }
+
+                @Override
+                public Set<Group> getGroups() {
+                    return null;
+                }
+            };
+        } else {
+            // a delegated provider contained a matching user
+            return userAndGroups;
+        }
+    }
+
+    @Override
+    public void preDestruction() throws AuthorizerDestructionException {
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardAuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardAuthorizableLookup.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardAuthorizableLookup.java
new file mode 100644
index 0000000..af48908
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardAuthorizableLookup.java
@@ -0,0 +1,239 @@
+/*
+ * 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.authorization;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.security.authorization.Resource;
+import org.apache.nifi.registry.security.authorization.resource.Authorizable;
+import org.apache.nifi.registry.security.authorization.resource.ResourceFactory;
+import org.apache.nifi.registry.security.authorization.resource.ResourceType;
+import org.apache.nifi.registry.security.authorization.resource.AccessPolicyAuthorizable;
+import org.apache.nifi.registry.exception.ResourceNotFoundException;
+import org.springframework.stereotype.Component;
+
+@Component
+public class StandardAuthorizableLookup implements AuthorizableLookup {
+
+    private static final Authorizable TENANTS_AUTHORIZABLE = new Authorizable() {
+        @Override
+        public Authorizable getParentAuthorizable() {
+            return null;
+        }
+
+        @Override
+        public Resource getResource() {
+            return ResourceFactory.getTenantResource();
+        }
+    };
+
+    private static final Authorizable POLICIES_AUTHORIZABLE = new Authorizable() {
+        @Override
+        public Authorizable getParentAuthorizable() {
+            return null;
+        }
+
+        @Override
+        public Resource getResource() {
+            return ResourceFactory.getPoliciesResource();
+        }
+    };
+
+    private static final Authorizable RESOURCES_AUTHORIZABLE = new Authorizable() {
+        @Override
+        public Authorizable getParentAuthorizable() {
+            return null;
+        }
+
+        @Override
+        public Resource getResource() {
+            return ResourceFactory.getResourceResource();
+        }
+    };
+
+    private static final Authorizable BUCKETS_AUTHORIZABLE = new Authorizable() {
+        @Override
+        public Authorizable getParentAuthorizable() {
+            return null;
+        }
+
+        @Override
+        public Resource getResource() {
+            return ResourceFactory.getBucketsResource();
+        }
+    };
+
+    private static final Authorizable PROXY_AUTHORIZABLE = new Authorizable() {
+        @Override
+        public Authorizable getParentAuthorizable() {
+            return null;
+        }
+
+        @Override
+        public Resource getResource() {
+            return ResourceFactory.getProxyResource();
+        }
+    };
+
+    @Override
+    public Authorizable getResourcesAuthorizable() {
+        return RESOURCES_AUTHORIZABLE;
+    }
+
+    @Override
+    public Authorizable getProxyAuthorizable() {
+        return PROXY_AUTHORIZABLE;
+    }
+
+    @Override
+    public Authorizable getTenantsAuthorizable() {
+        return TENANTS_AUTHORIZABLE;
+    }
+
+    @Override
+    public Authorizable getPoliciesAuthorizable() {
+        return POLICIES_AUTHORIZABLE;
+    }
+
+    @Override
+    public Authorizable getBucketsAuthorizable() {
+        return BUCKETS_AUTHORIZABLE;
+    }
+
+    @Override
+    public Authorizable getBucketAuthorizable(String bucketIdentifier) {
+        return new Authorizable() {
+
+            @Override
+            public Authorizable getParentAuthorizable() {
+                return getBucketsAuthorizable();
+            }
+
+            @Override
+            public Resource getResource() {
+                return ResourceFactory.getBucketResource(bucketIdentifier, null);
+            }
+        };
+    }
+
+    @Override
+    public Authorizable getAccessPolicyByResource(final String resource) {
+        try {
+            return new AccessPolicyAuthorizable(getAuthorizableByResource(resource));
+        } catch (final ResourceNotFoundException e) {
+            // the underlying component has been removed or resource is invalid... require /policies permissions
+            return POLICIES_AUTHORIZABLE;
+        }
+    }
+
+    @Override
+    public Authorizable getAuthorizableByResource(String resource) {
+        // parse the resource type
+        ResourceType resourceType = null;
+        for (ResourceType type : ResourceType.values()) {
+            if (resource.equals(type.getValue()) || resource.startsWith(type.getValue() + "/")) {
+                resourceType = type;
+            }
+        }
+
+        if (resourceType == null) {
+            throw new ResourceNotFoundException("Unrecognized resource: " + resource);
+        }
+
+        // if this is a policy resource, there should be another resource type
+        if (ResourceType.Policy.equals(resourceType)) {
+            final ResourceType primaryResourceType = resourceType;
+
+            // get the resource type
+            resource = StringUtils.substringAfter(resource, resourceType.getValue());
+
+            for (ResourceType type : ResourceType.values()) {
+                if (resource.equals(type.getValue()) || resource.startsWith(type.getValue() + "/")) {
+                    resourceType = type;
+                }
+            }
+
+            if (resourceType == null) {
+                throw new ResourceNotFoundException("Unrecognized resource: " + resource);
+            }
+
+            return new AccessPolicyAuthorizable(getAccessPolicy(resourceType, resource));
+        } else {
+            return getAccessPolicy(resourceType, resource);
+        }
+    }
+
+    private Authorizable getAccessPolicy(final ResourceType resourceType, final String resource) {
+        final String slashComponentId = StringUtils.substringAfter(resource, resourceType.getValue());
+        if (slashComponentId.startsWith("/")) {
+            return getAccessPolicyByResource(resourceType, slashComponentId.substring(1));
+        } else {
+            return getAccessPolicyByResource(resourceType);
+        }
+    }
+
+    private Authorizable getAccessPolicyByResource(final ResourceType resourceType, final String childResourceId) {
+        Authorizable authorizable = null;
+        switch (resourceType) {
+            case Bucket:
+                authorizable = getBucketAuthorizable(childResourceId);
+        }
+
+        if (authorizable == null) {
+            throw new IllegalArgumentException("An unexpected type of resource in this policy " + resourceType.getValue());
+        }
+
+        return authorizable;
+    }
+
+    private Authorizable getAccessPolicyByResource(final ResourceType resourceType) {
+        Authorizable authorizable = null;
+        switch (resourceType) {
+
+            case Bucket:
+                authorizable = getBucketsAuthorizable();
+                break;
+            case Policy:
+                authorizable = getPoliciesAuthorizable();
+                break;
+            case Resource:
+                authorizable = new Authorizable() {
+                    @Override
+                    public Authorizable getParentAuthorizable() {
+                        return null;
+                    }
+
+                    @Override
+                    public Resource getResource() {
+                        return ResourceFactory.getResourceResource();
+                    }
+                };
+                break;
+            case Tenant:
+                authorizable = getTenantsAuthorizable();
+                break;
+            case Proxy:
+                authorizable = getProxyAuthorizable();
+        }
+
+        if (authorizable == null) {
+            throw new IllegalArgumentException("An unexpected type of resource in this policy " + resourceType.getValue());
+        }
+
+        return authorizable;
+    }
+
+}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardAuthorizerInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardAuthorizerInitializationContext.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardAuthorizerInitializationContext.java
new file mode 100644
index 0000000..d643e91
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardAuthorizerInitializationContext.java
@@ -0,0 +1,52 @@
+/*
+ * 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.authorization;
+
+public class StandardAuthorizerInitializationContext implements AuthorizerInitializationContext {
+
+    private final String identifier;
+    private final UserGroupProviderLookup userGroupProviderLookup;
+    private final AccessPolicyProviderLookup accessPolicyProviderLookup;
+    private final AuthorizerLookup authorizerLookup;
+
+    public StandardAuthorizerInitializationContext(String identifier, UserGroupProviderLookup userGroupProviderLookup,
+                                                   AccessPolicyProviderLookup accessPolicyProviderLookup, AuthorizerLookup authorizerLookup) {
+        this.identifier = identifier;
+        this.userGroupProviderLookup = userGroupProviderLookup;
+        this.accessPolicyProviderLookup = accessPolicyProviderLookup;
+        this.authorizerLookup = authorizerLookup;
+    }
+
+    @Override
+    public String getIdentifier() {
+        return identifier;
+    }
+
+    public AuthorizerLookup getAuthorizerLookup() {
+        return authorizerLookup;
+    }
+
+    @Override
+    public AccessPolicyProviderLookup getAccessPolicyProviderLookup() {
+        return accessPolicyProviderLookup;
+    }
+
+    @Override
+    public UserGroupProviderLookup getUserGroupProviderLookup() {
+        return userGroupProviderLookup;
+    }
+}


[08/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAccessPolicyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAccessPolicyProvider.java b/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAccessPolicyProvider.java
deleted file mode 100644
index 47a81cc..0000000
--- a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAccessPolicyProvider.java
+++ /dev/null
@@ -1,757 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.file;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.util.PropertyValue;
-import org.apache.nifi.registry.authorization.annotation.AuthorizerContext;
-import org.apache.nifi.registry.authorization.file.generated.Authorizations;
-import org.apache.nifi.registry.authorization.file.generated.Policies;
-import org.apache.nifi.registry.authorization.file.generated.Policy;
-import org.apache.nifi.registry.properties.util.IdentityMapping;
-import org.apache.nifi.registry.properties.util.IdentityMappingUtil;
-import org.apache.nifi.registry.authorization.AccessPolicy;
-import org.apache.nifi.registry.authorization.AccessPolicyProviderInitializationContext;
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.AuthorizerConfigurationContext;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerDestructionException;
-import org.apache.nifi.registry.authorization.ConfigurableAccessPolicyProvider;
-import org.apache.nifi.registry.authorization.RequestAction;
-import org.apache.nifi.registry.authorization.exception.UninheritableAuthorizationsException;
-import org.apache.nifi.registry.authorization.User;
-import org.apache.nifi.registry.authorization.UserGroupProvider;
-import org.apache.nifi.registry.authorization.UserGroupProviderLookup;
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-
-import javax.xml.XMLConstants;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.stream.XMLOutputFactory;
-import javax.xml.stream.XMLStreamException;
-import javax.xml.stream.XMLStreamWriter;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.StringWriter;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Pattern;
-
-public class FileAccessPolicyProvider implements ConfigurableAccessPolicyProvider {
-
-    private static final Logger logger = LoggerFactory.getLogger(FileAccessPolicyProvider.class);
-
-    private static final String AUTHORIZATIONS_XSD = "/authorizations.xsd";
-    private static final String JAXB_AUTHORIZATIONS_PATH = "org.apache.nifi.registry.authorization.file.generated";
-
-    private static final JAXBContext JAXB_AUTHORIZATIONS_CONTEXT = initializeJaxbContext(JAXB_AUTHORIZATIONS_PATH);
-
-    /**
-     * Load the JAXBContext.
-     */
-    private static JAXBContext initializeJaxbContext(final String contextPath) {
-        try {
-            return JAXBContext.newInstance(contextPath, FileAuthorizer.class.getClassLoader());
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.");
-        }
-    }
-
-    private static final DocumentBuilderFactory DOCUMENT_BUILDER_FACTORY = DocumentBuilderFactory.newInstance();
-    private static final XMLOutputFactory XML_OUTPUT_FACTORY = XMLOutputFactory.newInstance();
-
-    private static final String POLICY_ELEMENT = "policy";
-    private static final String POLICY_USER_ELEMENT = "policyUser";
-    private static final String POLICY_GROUP_ELEMENT = "policyGroup";
-    private static final String IDENTIFIER_ATTR = "identifier";
-    private static final String RESOURCE_ATTR = "resource";
-    private static final String ACTIONS_ATTR = "actions";
-
-    /* These codes must match the enumeration values set in authorizations.xsd */
-    static final String READ_CODE = "R";
-    static final String WRITE_CODE = "W";
-    static final String DELETE_CODE = "D";
-
-    /*  TODO - move this somewhere into nifi-registry-security-framework so it can be applied to any ConfigurableAccessPolicyProvider
-     *  (and also gets us away from requiring magic strings here) */
-    private static final ResourceActionPair[] INITIAL_ADMIN_ACCESS_POLICIES = {
-            new ResourceActionPair("/resources", READ_CODE),
-            new ResourceActionPair("/tenants", READ_CODE),
-            new ResourceActionPair("/tenants", WRITE_CODE),
-            new ResourceActionPair("/tenants", DELETE_CODE),
-            new ResourceActionPair("/policies", READ_CODE),
-            new ResourceActionPair("/policies", WRITE_CODE),
-            new ResourceActionPair("/policies", DELETE_CODE),
-            new ResourceActionPair("/buckets", READ_CODE),
-            new ResourceActionPair("/buckets", WRITE_CODE),
-            new ResourceActionPair("/buckets", DELETE_CODE),
-            new ResourceActionPair("/proxy", WRITE_CODE)
-    };
-
-    static final String PROP_NODE_IDENTITY_PREFIX = "Node Identity ";
-    static final String PROP_USER_GROUP_PROVIDER = "User Group Provider";
-    static final String PROP_AUTHORIZATIONS_FILE = "Authorizations File";
-    static final String PROP_INITIAL_ADMIN_IDENTITY = "Initial Admin Identity";
-    static final Pattern NODE_IDENTITY_PATTERN = Pattern.compile(PROP_NODE_IDENTITY_PREFIX + "\\S+");
-
-    private Schema authorizationsSchema;
-    private NiFiRegistryProperties properties;
-    private File authorizationsFile;
-    private String initialAdminIdentity;
-    private List<IdentityMapping> identityMappings;
-
-    private UserGroupProvider userGroupProvider;
-    private UserGroupProviderLookup userGroupProviderLookup;
-    private final AtomicReference<AuthorizationsHolder> authorizationsHolder = new AtomicReference<>();
-
-    @Override
-    public void initialize(AccessPolicyProviderInitializationContext initializationContext) throws AuthorizerCreationException {
-        userGroupProviderLookup = initializationContext.getUserGroupProviderLookup();
-
-        try {
-            final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-            authorizationsSchema = schemaFactory.newSchema(FileAuthorizer.class.getResource(AUTHORIZATIONS_XSD));
-//            usersSchema = schemaFactory.newSchema(FileAuthorizer.class.getResource(USERS_XSD));
-        } catch (Exception e) {
-            throw new AuthorizerCreationException(e);
-        }
-    }
-
-    @Override
-    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-        try {
-            final PropertyValue userGroupProviderIdentifier = configurationContext.getProperty(PROP_USER_GROUP_PROVIDER);
-            if (!userGroupProviderIdentifier.isSet()) {
-                throw new AuthorizerCreationException("The user group provider must be specified.");
-            }
-
-            userGroupProvider = userGroupProviderLookup.getUserGroupProvider(userGroupProviderIdentifier.getValue());
-            if (userGroupProvider == null) {
-                throw new AuthorizerCreationException("Unable to locate user group provider with identifier " + userGroupProviderIdentifier.getValue());
-            }
-
-            final PropertyValue authorizationsPath = configurationContext.getProperty(PROP_AUTHORIZATIONS_FILE);
-            if (StringUtils.isBlank(authorizationsPath.getValue())) {
-                throw new AuthorizerCreationException("The authorizations file must be specified.");
-            }
-
-            // get the authorizations file and ensure it exists
-            authorizationsFile = new File(authorizationsPath.getValue());
-            if (!authorizationsFile.exists()) {
-                logger.info("Creating new authorizations file at {}", new Object[] {authorizationsFile.getAbsolutePath()});
-                saveAuthorizations(new Authorizations());
-            }
-
-            // extract the identity mappings from nifi.properties if any are provided
-            identityMappings = Collections.unmodifiableList(IdentityMappingUtil.getIdentityMappings(properties));
-
-            // get the value of the initial admin identity
-            final PropertyValue initialAdminIdentityProp = configurationContext.getProperty(PROP_INITIAL_ADMIN_IDENTITY);
-            initialAdminIdentity = initialAdminIdentityProp.isSet() ? IdentityMappingUtil.mapIdentity(initialAdminIdentityProp.getValue(), identityMappings) : null;
-
-//            // extract any node identities
-//            nodeIdentities = new HashSet<>();
-//            for (Map.Entry<String,String> entry : configurationContext.getProperties().entrySet()) {
-//                Matcher matcher = NODE_IDENTITY_PATTERN.matcher(entry.getKey());
-//                if (matcher.matches() && !StringUtils.isBlank(entry.getValue())) {
-//                    nodeIdentities.add(IdentityMappingUtil.mapIdentity(entry.getValue(), identityMappings));
-//                }
-//            }
-
-            // load the authorizations
-            load();
-
-            logger.info(String.format("Authorizations file loaded at %s", new Date().toString()));
-        } catch (AuthorizerCreationException | JAXBException | IllegalStateException | SAXException e) {
-            throw new AuthorizerCreationException(e);
-        }
-    }
-
-    @Override
-    public UserGroupProvider getUserGroupProvider() {
-        return userGroupProvider;
-    }
-
-    @Override
-    public Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException {
-        return authorizationsHolder.get().getAllPolicies();
-    }
-
-    @Override
-    public synchronized AccessPolicy addAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
-        if (accessPolicy == null) {
-            throw new IllegalArgumentException("AccessPolicy cannot be null");
-        }
-
-        // create the new JAXB Policy
-        final Policy policy = createJAXBPolicy(accessPolicy);
-
-        // add the new Policy to the top-level list of policies
-        final AuthorizationsHolder holder = authorizationsHolder.get();
-        final Authorizations authorizations = holder.getAuthorizations();
-        authorizations.getPolicies().getPolicy().add(policy);
-
-        saveAndRefreshHolder(authorizations);
-
-        return authorizationsHolder.get().getPoliciesById().get(accessPolicy.getIdentifier());
-    }
-
-    @Override
-    public AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException {
-        if (identifier == null) {
-            return null;
-        }
-
-        final AuthorizationsHolder holder = authorizationsHolder.get();
-        return holder.getPoliciesById().get(identifier);
-    }
-
-    @Override
-    public AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) throws AuthorizationAccessException {
-        return authorizationsHolder.get().getAccessPolicy(resourceIdentifier, action);
-    }
-
-    @Override
-    public synchronized AccessPolicy updateAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
-        if (accessPolicy == null) {
-            throw new IllegalArgumentException("AccessPolicy cannot be null");
-        }
-
-        final AuthorizationsHolder holder = this.authorizationsHolder.get();
-        final Authorizations authorizations = holder.getAuthorizations();
-
-        // try to find an existing Authorization that matches the policy id
-        Policy updatePolicy = null;
-        for (Policy policy : authorizations.getPolicies().getPolicy()) {
-            if (policy.getIdentifier().equals(accessPolicy.getIdentifier())) {
-                updatePolicy = policy;
-                break;
-            }
-        }
-
-        // no matching Policy so return null
-        if (updatePolicy == null) {
-            return null;
-        }
-
-        // update the Policy, save, reload, and return
-        transferUsersAndGroups(accessPolicy, updatePolicy);
-        saveAndRefreshHolder(authorizations);
-
-        return this.authorizationsHolder.get().getPoliciesById().get(accessPolicy.getIdentifier());
-    }
-
-    @Override
-    public synchronized AccessPolicy deleteAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
-        if (accessPolicy == null) {
-            throw new IllegalArgumentException("AccessPolicy cannot be null");
-        }
-
-        return deleteAccessPolicy(accessPolicy.getIdentifier());
-    }
-
-    @Override
-    public synchronized AccessPolicy deleteAccessPolicy(String accessPolicyIdentifer) throws AuthorizationAccessException {
-        if (accessPolicyIdentifer == null) {
-            throw new IllegalArgumentException("Access policy identifier cannot be null");
-        }
-
-        final AuthorizationsHolder holder = this.authorizationsHolder.get();
-        AccessPolicy deletedPolicy = holder.getPoliciesById().get(accessPolicyIdentifer);
-        if (deletedPolicy == null) {
-            return null;
-        }
-
-        // find the matching Policy and remove it
-        final Authorizations authorizations = holder.getAuthorizations();
-        Iterator<Policy> policyIter = authorizations.getPolicies().getPolicy().iterator();
-        while (policyIter.hasNext()) {
-            final Policy policy = policyIter.next();
-            if (policy.getIdentifier().equals(accessPolicyIdentifer)) {
-                policyIter.remove();
-                break;
-            }
-        }
-
-        saveAndRefreshHolder(authorizations);
-        return deletedPolicy;
-    }
-
-    AuthorizationsHolder getAuthorizationsHolder() {
-        return authorizationsHolder.get();
-    }
-
-    @AuthorizerContext
-    public void setNiFiProperties(NiFiRegistryProperties properties) {
-        this.properties = properties;
-    }
-
-    @Override
-    public synchronized void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
-        parsePolicies(fingerprint).forEach(policy -> addAccessPolicy(policy));
-    }
-
-    @Override
-    public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
-        try {
-            // ensure we can understand the proposed fingerprint
-            parsePolicies(proposedFingerprint);
-        } catch (final AuthorizationAccessException e) {
-            throw new UninheritableAuthorizationsException("Unable to parse the proposed fingerprint: " + e);
-        }
-
-        // ensure we are in a proper state to inherit the fingerprint
-        if (!getAccessPolicies().isEmpty()) {
-            throw new UninheritableAuthorizationsException("Proposed fingerprint is not inheritable because the current access policies is not empty.");
-        }
-    }
-
-    @Override
-    public String getFingerprint() throws AuthorizationAccessException {
-        final List<AccessPolicy> policies = new ArrayList<>(getAccessPolicies());
-        Collections.sort(policies, Comparator.comparing(AccessPolicy::getIdentifier));
-
-        XMLStreamWriter writer = null;
-        final StringWriter out = new StringWriter();
-        try {
-            writer = XML_OUTPUT_FACTORY.createXMLStreamWriter(out);
-            writer.writeStartDocument();
-            writer.writeStartElement("accessPolicies");
-
-            for (AccessPolicy policy : policies) {
-                writePolicy(writer, policy);
-            }
-
-            writer.writeEndElement();
-            writer.writeEndDocument();
-            writer.flush();
-        } catch (XMLStreamException e) {
-            throw new AuthorizationAccessException("Unable to generate fingerprint", e);
-        } finally {
-            if (writer != null) {
-                try {
-                    writer.close();
-                } catch (XMLStreamException e) {
-                    // nothing to do here
-                }
-            }
-        }
-
-        return out.toString();
-    }
-
-    private List<AccessPolicy> parsePolicies(final String fingerprint) {
-        final List<AccessPolicy> policies = new ArrayList<>();
-
-        final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
-        try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
-            final DocumentBuilder docBuilder = DOCUMENT_BUILDER_FACTORY.newDocumentBuilder();
-            final Document document = docBuilder.parse(in);
-            final Element rootElement = document.getDocumentElement();
-
-            // parse all the policies and add them to the current access policy provider
-            NodeList policyNodes = rootElement.getElementsByTagName(POLICY_ELEMENT);
-            for (int i = 0; i < policyNodes.getLength(); i++) {
-                Node policyNode = policyNodes.item(i);
-                policies.add(parsePolicy((Element) policyNode));
-            }
-        } catch (SAXException | ParserConfigurationException | IOException e) {
-            throw new AuthorizationAccessException("Unable to parse fingerprint", e);
-        }
-
-        return policies;
-    }
-
-    private AccessPolicy parsePolicy(final Element element) {
-        final AccessPolicy.Builder builder = new AccessPolicy.Builder()
-                .identifier(element.getAttribute(IDENTIFIER_ATTR))
-                .resource(element.getAttribute(RESOURCE_ATTR));
-
-        final String actions = element.getAttribute(ACTIONS_ATTR);
-        if (actions.equals(RequestAction.READ.name())) {
-            builder.action(RequestAction.READ);
-        } else if (actions.equals(RequestAction.WRITE.name())) {
-            builder.action(RequestAction.WRITE);
-        } else if (actions.equals(RequestAction.DELETE.name())) {
-            builder.action(RequestAction.DELETE);
-        } else {
-            throw new IllegalStateException("Unknown Policy Action: " + actions);
-        }
-
-        NodeList policyUsers = element.getElementsByTagName(POLICY_USER_ELEMENT);
-        for (int i=0; i < policyUsers.getLength(); i++) {
-            Element policyUserNode = (Element) policyUsers.item(i);
-            builder.addUser(policyUserNode.getAttribute(IDENTIFIER_ATTR));
-        }
-
-        NodeList policyGroups = element.getElementsByTagName(POLICY_GROUP_ELEMENT);
-        for (int i=0; i < policyGroups.getLength(); i++) {
-            Element policyGroupNode = (Element) policyGroups.item(i);
-            builder.addGroup(policyGroupNode.getAttribute(IDENTIFIER_ATTR));
-        }
-
-        return builder.build();
-    }
-
-    private void writePolicy(final XMLStreamWriter writer, final AccessPolicy policy) throws XMLStreamException {
-        // sort the users for the policy
-        List<String> policyUsers = new ArrayList<>(policy.getUsers());
-        Collections.sort(policyUsers);
-
-        // sort the groups for this policy
-        List<String> policyGroups = new ArrayList<>(policy.getGroups());
-        Collections.sort(policyGroups);
-
-        writer.writeStartElement(POLICY_ELEMENT);
-        writer.writeAttribute(IDENTIFIER_ATTR, policy.getIdentifier());
-        writer.writeAttribute(RESOURCE_ATTR, policy.getResource());
-        writer.writeAttribute(ACTIONS_ATTR, policy.getAction().name());
-
-        for (String policyUser : policyUsers) {
-            writer.writeStartElement(POLICY_USER_ELEMENT);
-            writer.writeAttribute(IDENTIFIER_ATTR, policyUser);
-            writer.writeEndElement();
-        }
-
-        for (String policyGroup : policyGroups) {
-            writer.writeStartElement(POLICY_GROUP_ELEMENT);
-            writer.writeAttribute(IDENTIFIER_ATTR, policyGroup);
-            writer.writeEndElement();
-        }
-
-        writer.writeEndElement();
-    }
-
-    /**
-     * Loads the authorizations file and populates the AuthorizationsHolder, only called during start-up.
-     *
-     * @throws JAXBException            Unable to reload the authorized users file
-     */
-    private synchronized void load() throws JAXBException, SAXException {
-        // attempt to unmarshal
-        final Authorizations authorizations = unmarshallAuthorizations();
-        if (authorizations.getPolicies() == null) {
-            authorizations.setPolicies(new Policies());
-        }
-
-        final AuthorizationsHolder authorizationsHolder = new AuthorizationsHolder(authorizations);
-        final boolean emptyAuthorizations = authorizationsHolder.getAllPolicies().isEmpty();
-        final boolean hasInitialAdminIdentity = (initialAdminIdentity != null && !StringUtils.isBlank(initialAdminIdentity));
-
-        // if we are starting fresh then we might need to populate an initial admin
-        if (emptyAuthorizations && hasInitialAdminIdentity) {
-            logger.info("Populating authorizations for Initial Admin: " + initialAdminIdentity);
-            populateInitialAdmin(authorizations);
-            saveAndRefreshHolder(authorizations);
-        } else {
-            this.authorizationsHolder.set(authorizationsHolder);
-        }
-    }
-
-    private void saveAuthorizations(final Authorizations authorizations) throws JAXBException {
-        final Marshaller marshaller = JAXB_AUTHORIZATIONS_CONTEXT.createMarshaller();
-        marshaller.setSchema(authorizationsSchema);
-        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
-        marshaller.marshal(authorizations, authorizationsFile);
-    }
-
-    private Authorizations unmarshallAuthorizations() throws JAXBException {
-        final Unmarshaller unmarshaller = JAXB_AUTHORIZATIONS_CONTEXT.createUnmarshaller();
-        unmarshaller.setSchema(authorizationsSchema);
-
-        final JAXBElement<Authorizations> element = unmarshaller.unmarshal(new StreamSource(authorizationsFile), Authorizations.class);
-        return element.getValue();
-    }
-
-    /**
-     *  Creates the initial admin user and sets policies managing buckets, users, and policies.
-     *
-     *  TODO - move this somewhere into nifi-registry-security-framework so it can be applied to any ConfigurableAccessPolicyProvider
-     */
-    private void populateInitialAdmin(final Authorizations authorizations) {
-        final User initialAdmin = userGroupProvider.getUserByIdentity(initialAdminIdentity);
-        if (initialAdmin == null) {
-            throw new AuthorizerCreationException("Unable to locate initial admin " + initialAdminIdentity + " to seed policies");
-        }
-
-        for (ResourceActionPair resourceAction : INITIAL_ADMIN_ACCESS_POLICIES) {
-            addUserToAccessPolicy(authorizations, resourceAction.resource, initialAdmin.getIdentifier(), resourceAction.actionCode);
-        }
-    }
-
-//    /**
-//     * Creates a user for each node and gives the nodes write permission to /proxy.
-//     *
-//     * @param authorizations the overall authorizations
-//     */
-//    private void populateNodes(Authorizations authorizations) {
-//        for (String nodeIdentity : nodeIdentities) {
-//            final User node = userGroupProvider.getUserByIdentity(nodeIdentity);
-//            if (node == null) {
-//                throw new AuthorizerCreationException("Unable to locate node " + nodeIdentity + " to seed policies.");
-//            }
-//
-//            // grant access to the proxy resource
-//            addUserToAccessPolicy(authorizations, ResourceType.Proxy.getValue(), node.getIdentifier(), WRITE_CODE);
-//
-//            // grant the user read/write access data of the root group
-//            if (rootGroupId != null) {
-//                addUserToAccessPolicy(authorizations, ResourceType.Data.getValue() + ResourceType.ProcessGroup.getValue() + "/" + rootGroupId, node.getIdentifier(), READ_CODE);
-//                addUserToAccessPolicy(authorizations, ResourceType.Data.getValue() + ResourceType.ProcessGroup.getValue() + "/" + rootGroupId, node.getIdentifier(), WRITE_CODE);
-//            }
-//        }
-//    }
-
-
-    /**
-     * Creates and adds an access policy for the given resource, identity, and actions to the specified authorizations.
-     *
-     * @param authorizations the Authorizations instance to add the policy to
-     * @param resource the resource for the policy
-     * @param userIdentifier the identifier for the user to add to the policy
-     * @param action the action for the policy
-     */
-    private void addUserToAccessPolicy(final Authorizations authorizations, final String resource, final String userIdentifier, final String action) {
-        // first try to find an existing policy for the given resource and action
-        Policy foundPolicy = null;
-        for (Policy policy : authorizations.getPolicies().getPolicy()) {
-            if (policy.getResource().equals(resource) && policy.getAction().equals(action)) {
-                foundPolicy = policy;
-                break;
-            }
-        }
-
-        if (foundPolicy == null) {
-            // if we didn't find an existing policy create a new one
-            final String uuidSeed = resource + action;
-
-            final AccessPolicy.Builder builder = new AccessPolicy.Builder()
-                    .identifierGenerateFromSeed(uuidSeed)
-                    .resource(resource)
-                    .addUser(userIdentifier);
-
-            if (action.equals(READ_CODE)) {
-                builder.action(RequestAction.READ);
-            } else if (action.equals(WRITE_CODE)) {
-                builder.action(RequestAction.WRITE);
-            } else if (action.equals(DELETE_CODE)) {
-                builder.action(RequestAction.DELETE);
-            } else {
-                throw new IllegalStateException("Unknown Policy Action: " + action);
-            }
-
-            final AccessPolicy accessPolicy = builder.build();
-            final Policy jaxbPolicy = createJAXBPolicy(accessPolicy);
-            authorizations.getPolicies().getPolicy().add(jaxbPolicy);
-        } else {
-            // otherwise add the user to the existing policy
-            Policy.User policyUser = new Policy.User();
-            policyUser.setIdentifier(userIdentifier);
-            foundPolicy.getUser().add(policyUser);
-        }
-    }
-
-    private Policy createJAXBPolicy(final AccessPolicy accessPolicy) {
-        final Policy policy = new Policy();
-        policy.setIdentifier(accessPolicy.getIdentifier());
-        policy.setResource(accessPolicy.getResource());
-
-        switch (accessPolicy.getAction()) {
-            case READ:
-                policy.setAction(READ_CODE);
-                break;
-            case WRITE:
-                policy.setAction(WRITE_CODE);
-                break;
-            case DELETE:
-                policy.setAction(DELETE_CODE);
-                break;
-            default:
-                break;
-        }
-
-        transferUsersAndGroups(accessPolicy, policy);
-        return policy;
-    }
-
-    /**
-     * Sets the given Policy to the state of the provided AccessPolicy. Users and Groups will be cleared and
-     * set to match the AccessPolicy, the resource and action will be set to match the AccessPolicy.
-     *
-     * Does not set the identifier.
-     *
-     * @param accessPolicy the AccessPolicy to transfer state from
-     * @param policy the Policy to transfer state to
-     */
-    private void transferUsersAndGroups(AccessPolicy accessPolicy, Policy policy) {
-        // add users to the policy
-        policy.getUser().clear();
-        for (String userIdentifier : accessPolicy.getUsers()) {
-            Policy.User policyUser = new Policy.User();
-            policyUser.setIdentifier(userIdentifier);
-            policy.getUser().add(policyUser);
-        }
-
-        // add groups to the policy
-        policy.getGroup().clear();
-        for (String groupIdentifier : accessPolicy.getGroups()) {
-            Policy.Group policyGroup = new Policy.Group();
-            policyGroup.setIdentifier(groupIdentifier);
-            policy.getGroup().add(policyGroup);
-        }
-    }
-
-    /**
-     * Adds the given user identifier to the policy if it doesn't already exist.
-     *
-     * @param userIdentifier a user identifier
-     * @param policy a policy to add the user to
-     */
-    private void addUserToPolicy(final String userIdentifier, final Policy policy) {
-        // determine if the user already exists in the policy
-        boolean userExists = false;
-        for (Policy.User policyUser : policy.getUser()) {
-            if (policyUser.getIdentifier().equals(userIdentifier)) {
-                userExists = true;
-                break;
-            }
-        }
-
-        // add the user to the policy if doesn't already exist
-        if (!userExists) {
-            Policy.User policyUser = new Policy.User();
-            policyUser.setIdentifier(userIdentifier);
-            policy.getUser().add(policyUser);
-        }
-    }
-
-    /**
-     * Adds the given group identifier to the policy if it doesn't already exist.
-     *
-     * @param groupIdentifier a group identifier
-     * @param policy a policy to add the user to
-     */
-    private void addGroupToPolicy(final String groupIdentifier, final Policy policy) {
-        // determine if the group already exists in the policy
-        boolean groupExists = false;
-        for (Policy.Group policyGroup : policy.getGroup()) {
-            if (policyGroup.getIdentifier().equals(groupIdentifier)) {
-                groupExists = true;
-                break;
-            }
-        }
-
-        // add the group to the policy if doesn't already exist
-        if (!groupExists) {
-            Policy.Group policyGroup = new Policy.Group();
-            policyGroup.setIdentifier(groupIdentifier);
-            policy.getGroup().add(policyGroup);
-        }
-    }
-
-    /**
-     * Finds the Policy matching the resource and action, or creates a new one and adds it to the list of policies.
-     *
-     * @param policies the policies to search through
-     * @param seedIdentity the seedIdentity to use when creating identifiers for new policies
-     * @param resource the resource for the policy
-     * @param action the action string for the police (R or RW)
-     * @return the matching policy or a new policy
-     */
-    private Policy getOrCreatePolicy(final List<Policy> policies, final String seedIdentity, final String resource, final String action) {
-        Policy foundPolicy = null;
-
-        // try to find a policy with the same resource and actions
-        for (Policy policy : policies) {
-            if (policy.getResource().equals(resource) && policy.getAction().equals(action)) {
-                foundPolicy = policy;
-                break;
-            }
-        }
-
-        // if a matching policy wasn't found then create one
-        if (foundPolicy == null) {
-            final String uuidSeed = resource + action + seedIdentity;
-            final String policyIdentifier = IdentifierUtil.getIdentifier(uuidSeed);
-
-            foundPolicy = new Policy();
-            foundPolicy.setIdentifier(policyIdentifier);
-            foundPolicy.setResource(resource);
-            foundPolicy.setAction(action);
-
-            policies.add(foundPolicy);
-        }
-
-        return foundPolicy;
-    }
-
-    /**
-     * Saves the Authorizations instance by marshalling to a file, then re-populates the
-     * in-memory data structures and sets the new holder.
-     *
-     * Synchronized to ensure only one thread writes the file at a time.
-     *
-     * @param authorizations the authorizations to save and populate from
-     * @throws AuthorizationAccessException if an error occurs saving the authorizations
-     */
-    private synchronized void saveAndRefreshHolder(final Authorizations authorizations) throws AuthorizationAccessException {
-        try {
-            saveAuthorizations(authorizations);
-
-            this.authorizationsHolder.set(new AuthorizationsHolder(authorizations));
-        } catch (JAXBException e) {
-            throw new AuthorizationAccessException("Unable to save Authorizations", e);
-        }
-    }
-
-    @Override
-    public void preDestruction() throws AuthorizerDestructionException {
-    }
-
-    private static class ResourceActionPair {
-        public String resource;
-        public String actionCode;
-        public ResourceActionPair(String resource, String actionCode) {
-            this.resource = resource;
-            this.actionCode = actionCode;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAuthorizer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAuthorizer.java b/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAuthorizer.java
deleted file mode 100644
index ef273d9..0000000
--- a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileAuthorizer.java
+++ /dev/null
@@ -1,288 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.file;
-
-import org.apache.nifi.registry.authorization.StandardAuthorizerConfigurationContext;
-import org.apache.nifi.registry.authorization.annotation.AuthorizerContext;
-import org.apache.nifi.registry.authorization.AbstractPolicyBasedAuthorizer;
-import org.apache.nifi.registry.authorization.AccessPolicy;
-import org.apache.nifi.registry.authorization.AccessPolicyProviderInitializationContext;
-import org.apache.nifi.registry.authorization.AccessPolicyProviderLookup;
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.AuthorizerConfigurationContext;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.AuthorizerInitializationContext;
-import org.apache.nifi.registry.authorization.Group;
-import org.apache.nifi.registry.authorization.RequestAction;
-import org.apache.nifi.registry.authorization.User;
-import org.apache.nifi.registry.authorization.UserGroupProviderInitializationContext;
-import org.apache.nifi.registry.authorization.UserGroupProviderLookup;
-import org.apache.nifi.registry.authorization.UsersAndAccessPolicies;
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Matcher;
-
-/**
- * Provides authorizes requests to resources using policies persisted in a file.
- */
-public class FileAuthorizer extends AbstractPolicyBasedAuthorizer {
-
-    private static final Logger logger = LoggerFactory.getLogger(FileAuthorizer.class);
-
-    private static final String FILE_USER_GROUP_PROVIDER_ID = "file-user-group-provider";
-    private static final String FILE_ACCESS_POLICY_PROVIDER_ID = "file-access-policy-provider";
-
-    static final String PROP_LEGACY_AUTHORIZED_USERS_FILE = "Legacy Authorized Users File";
-
-    private FileUserGroupProvider userGroupProvider = new FileUserGroupProvider();
-    private FileAccessPolicyProvider accessPolicyProvider = new FileAccessPolicyProvider();
-
-    @Override
-    public void initialize(final AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
-        // initialize the user group provider
-        userGroupProvider.initialize(new UserGroupProviderInitializationContext() {
-            @Override
-            public String getIdentifier() {
-                return FILE_USER_GROUP_PROVIDER_ID;
-            }
-
-            @Override
-            public UserGroupProviderLookup getUserGroupProviderLookup() {
-                return (identifier) -> null;
-            }
-        });
-
-        // initialize the access policy provider
-        accessPolicyProvider.initialize(new AccessPolicyProviderInitializationContext() {
-            @Override
-            public String getIdentifier() {
-                return FILE_ACCESS_POLICY_PROVIDER_ID;
-            }
-
-            @Override
-            public UserGroupProviderLookup getUserGroupProviderLookup() {
-                return (identifier) -> {
-                    if (FILE_USER_GROUP_PROVIDER_ID.equals(identifier)) {
-                        return userGroupProvider;
-                    }
-
-                    return null;
-                };
-            }
-
-            @Override
-            public AccessPolicyProviderLookup getAccessPolicyProviderLookup() {
-                return (identifier) ->  null;
-            }
-        });
-    }
-
-    @Override
-    public void doOnConfigured(final AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-        final Map<String, String> configurationProperties = configurationContext.getProperties();
-
-        // relay the relevant config
-        final Map<String, String> userGroupProperties = new HashMap<>();
-        if (configurationProperties.containsKey(FileUserGroupProvider.PROP_TENANTS_FILE)) {
-            userGroupProperties.put(FileUserGroupProvider.PROP_TENANTS_FILE, configurationProperties.get(FileUserGroupProvider.PROP_TENANTS_FILE));
-        }
-        if (configurationProperties.containsKey(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)) {
-            userGroupProperties.put(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE, configurationProperties.get(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE));
-        }
-
-        // relay the relevant config
-        final Map<String, String> accessPolicyProperties = new HashMap<>();
-        accessPolicyProperties.put(FileAccessPolicyProvider.PROP_USER_GROUP_PROVIDER, FILE_USER_GROUP_PROVIDER_ID);
-        if (configurationProperties.containsKey(FileAccessPolicyProvider.PROP_AUTHORIZATIONS_FILE)) {
-            accessPolicyProperties.put(FileAccessPolicyProvider.PROP_AUTHORIZATIONS_FILE, configurationProperties.get(FileAccessPolicyProvider.PROP_AUTHORIZATIONS_FILE));
-        }
-        if (configurationProperties.containsKey(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)) {
-            accessPolicyProperties.put(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY, configurationProperties.get(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY));
-        }
-        if (configurationProperties.containsKey(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)) {
-            accessPolicyProperties.put(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE, configurationProperties.get(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE));
-        }
-
-        // ensure all node identities are seeded into the user provider
-        configurationProperties.forEach((property, value) -> {
-            final Matcher matcher = FileAccessPolicyProvider.NODE_IDENTITY_PATTERN.matcher(property);
-            if (matcher.matches()) {
-                accessPolicyProperties.put(property, value);
-                userGroupProperties.put(property.replace(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX, FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX), value);
-            }
-        });
-
-        // ensure the initial admin is seeded into the user provider if appropriate
-        if (configurationProperties.containsKey(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)) {
-            int i = 0;
-            while (true) {
-                final String key = FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + i++;
-                if (!userGroupProperties.containsKey(key)) {
-                    userGroupProperties.put(key, configurationProperties.get(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY));
-                    break;
-                }
-            }
-        }
-
-        // configure the user group provider
-        userGroupProvider.onConfigured(new StandardAuthorizerConfigurationContext(FILE_USER_GROUP_PROVIDER_ID, userGroupProperties));
-
-        // configure the access policy provider
-        accessPolicyProvider.onConfigured(new StandardAuthorizerConfigurationContext(FILE_USER_GROUP_PROVIDER_ID, accessPolicyProperties));
-    }
-
-    @Override
-    public void preDestruction() {
-
-    }
-
-    // ------------------ Groups ------------------
-
-    @Override
-    public synchronized Group doAddGroup(Group group) throws AuthorizationAccessException {
-        return userGroupProvider.addGroup(group);
-    }
-
-    @Override
-    public Group getGroup(String identifier) throws AuthorizationAccessException {
-        return userGroupProvider.getGroup(identifier);
-    }
-
-    @Override
-    public synchronized Group doUpdateGroup(Group group) throws AuthorizationAccessException {
-        return userGroupProvider.updateGroup(group);
-    }
-
-    @Override
-    public synchronized Group deleteGroup(Group group) throws AuthorizationAccessException {
-        return userGroupProvider.deleteGroup(group);
-    }
-
-    @Override
-    public synchronized Group deleteGroup(String groupId) throws AuthorizationAccessException {
-        return userGroupProvider.deleteGroup(groupId);
-    }
-
-    @Override
-    public Set<Group> getGroups() throws AuthorizationAccessException {
-        return userGroupProvider.getGroups();
-    }
-
-    // ------------------ Users ------------------
-
-    @Override
-    public synchronized User doAddUser(final User user) throws AuthorizationAccessException {
-        return userGroupProvider.addUser(user);
-    }
-
-    @Override
-    public User getUser(final String identifier) throws AuthorizationAccessException {
-        return userGroupProvider.getUser(identifier);
-    }
-
-    @Override
-    public User getUserByIdentity(final String identity) throws AuthorizationAccessException {
-        return userGroupProvider.getUserByIdentity(identity);
-    }
-
-    @Override
-    public synchronized User doUpdateUser(final User user) throws AuthorizationAccessException {
-        return userGroupProvider.updateUser(user);
-    }
-
-    @Override
-    public synchronized User deleteUser(final User user) throws AuthorizationAccessException {
-        return userGroupProvider.deleteUser(user);
-    }
-
-    @Override
-    public synchronized User deleteUser(final String userId) throws AuthorizationAccessException {
-        return userGroupProvider.deleteUser(userId);
-    }
-
-    @Override
-    public Set<User> getUsers() throws AuthorizationAccessException {
-        return userGroupProvider.getUsers();
-    }
-
-    // ------------------ AccessPolicies ------------------
-
-    @Override
-    public synchronized AccessPolicy doAddAccessPolicy(final AccessPolicy accessPolicy) throws AuthorizationAccessException {
-        return accessPolicyProvider.addAccessPolicy(accessPolicy);
-    }
-
-    @Override
-    public AccessPolicy getAccessPolicy(final String identifier) throws AuthorizationAccessException {
-        return accessPolicyProvider.getAccessPolicy(identifier);
-    }
-
-    @Override
-    public synchronized AccessPolicy updateAccessPolicy(final AccessPolicy accessPolicy) throws AuthorizationAccessException {
-        return accessPolicyProvider.updateAccessPolicy(accessPolicy);
-    }
-
-    @Override
-    public synchronized AccessPolicy deleteAccessPolicy(final AccessPolicy accessPolicy) throws AuthorizationAccessException {
-        return accessPolicyProvider.deleteAccessPolicy(accessPolicy);
-    }
-
-    @Override
-    public synchronized AccessPolicy deleteAccessPolicy(final String accessPolicyIdentifier) throws AuthorizationAccessException {
-        return accessPolicyProvider.deleteAccessPolicy(accessPolicyIdentifier);
-    }
-
-    @Override
-    public Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException {
-        return accessPolicyProvider.getAccessPolicies();
-    }
-
-    @AuthorizerContext
-    public void setNiFiProperties(NiFiRegistryProperties properties) {
-        userGroupProvider.setNiFiProperties(properties);
-        accessPolicyProvider.setNiFiProperties(properties);
-    }
-
-    @Override
-    public synchronized UsersAndAccessPolicies getUsersAndAccessPolicies() throws AuthorizationAccessException {
-        final AuthorizationsHolder authorizationsHolder = accessPolicyProvider.getAuthorizationsHolder();
-        final UserGroupHolder userGroupHolder = userGroupProvider.getUserGroupHolder();
-
-        return new UsersAndAccessPolicies() {
-            @Override
-            public AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) {
-                return authorizationsHolder.getAccessPolicy(resourceIdentifier, action);
-            }
-
-            @Override
-            public User getUser(String identity) {
-                return userGroupHolder.getUser(identity);
-            }
-
-            @Override
-            public Set<Group> getGroups(String userIdentity) {
-                return userGroupHolder.getGroups(userIdentity);
-            }
-        };
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileUserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileUserGroupProvider.java b/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileUserGroupProvider.java
deleted file mode 100644
index 54c3d96..0000000
--- a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/FileUserGroupProvider.java
+++ /dev/null
@@ -1,775 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.file;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.authorization.annotation.AuthorizerContext;
-import org.apache.nifi.registry.authorization.file.tenants.generated.Groups;
-import org.apache.nifi.registry.authorization.file.tenants.generated.Tenants;
-import org.apache.nifi.registry.authorization.file.tenants.generated.Users;
-import org.apache.nifi.registry.properties.util.IdentityMapping;
-import org.apache.nifi.registry.properties.util.IdentityMappingUtil;
-import org.apache.nifi.registry.util.PropertyValue;
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.AuthorizerConfigurationContext;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerDestructionException;
-import org.apache.nifi.registry.authorization.ConfigurableUserGroupProvider;
-import org.apache.nifi.registry.authorization.Group;
-import org.apache.nifi.registry.authorization.exception.UninheritableAuthorizationsException;
-import org.apache.nifi.registry.authorization.User;
-import org.apache.nifi.registry.authorization.UserAndGroups;
-import org.apache.nifi.registry.authorization.UserGroupProviderInitializationContext;
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.util.FileUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-
-import javax.xml.XMLConstants;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Marshaller;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.stream.XMLOutputFactory;
-import javax.xml.stream.XMLStreamException;
-import javax.xml.stream.XMLStreamWriter;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.IOException;
-import java.io.StringWriter;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-public class FileUserGroupProvider implements ConfigurableUserGroupProvider {
-
-    private static final Logger logger = LoggerFactory.getLogger(FileUserGroupProvider.class);
-
-    private static final String TENANTS_XSD = "/tenants.xsd";
-    private static final String JAXB_TENANTS_PATH = "org.apache.nifi.registry.authorization.file.tenants.generated";
-
-    private static final JAXBContext JAXB_TENANTS_CONTEXT = initializeJaxbContext(JAXB_TENANTS_PATH);
-
-    /**
-     * Load the JAXBContext.
-     */
-    private static JAXBContext initializeJaxbContext(final String contextPath) {
-        try {
-            return JAXBContext.newInstance(contextPath, FileAuthorizer.class.getClassLoader());
-            //return JAXBContext.newInstance(contextPath);
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext: " + e);
-        }
-    }
-
-    private static final DocumentBuilderFactory DOCUMENT_BUILDER_FACTORY = DocumentBuilderFactory.newInstance();
-    private static final XMLOutputFactory XML_OUTPUT_FACTORY = XMLOutputFactory.newInstance();
-
-    private static final String USER_ELEMENT = "user";
-    private static final String GROUP_USER_ELEMENT = "groupUser";
-    private static final String GROUP_ELEMENT = "group";
-    private static final String IDENTIFIER_ATTR = "identifier";
-    private static final String IDENTITY_ATTR = "identity";
-    private static final String NAME_ATTR = "name";
-
-    static final String PROP_INITIAL_USER_IDENTITY_PREFIX = "Initial User Identity ";
-    static final String PROP_TENANTS_FILE = "Users File";
-    static final Pattern INITIAL_USER_IDENTITY_PATTERN = Pattern.compile(PROP_INITIAL_USER_IDENTITY_PREFIX + "\\S+");
-
-    private Schema usersSchema;
-    private Schema tenantsSchema;
-    private NiFiRegistryProperties properties;
-    private File tenantsFile;
-    private File restoreTenantsFile;
-    private Set<String> initialUserIdentities;
-    private List<IdentityMapping> identityMappings;
-
-    private final AtomicReference<UserGroupHolder> userGroupHolder = new AtomicReference<>();
-
-    @Override
-    public void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
-        try {
-            final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-            tenantsSchema = schemaFactory.newSchema(FileAuthorizer.class.getResource(TENANTS_XSD));
-            //usersSchema = schemaFactory.newSchema(FileAuthorizer.class.getResource(USERS_XSD));
-        } catch (Exception e) {
-            throw new AuthorizerCreationException(e);
-        }
-    }
-
-    @Override
-    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-        try {
-            final PropertyValue tenantsPath = configurationContext.getProperty(PROP_TENANTS_FILE);
-            if (StringUtils.isBlank(tenantsPath.getValue())) {
-                throw new AuthorizerCreationException("The users file must be specified.");
-            }
-
-            // get the tenants file and ensure it exists
-            tenantsFile = new File(tenantsPath.getValue());
-            if (!tenantsFile.exists()) {
-                logger.info("Creating new users file at {}", new Object[] {tenantsFile.getAbsolutePath()});
-                saveTenants(new Tenants());
-            }
-
-            final File tenantsFileDirectory = tenantsFile.getAbsoluteFile().getParentFile();
-
-            // extract the identity mappings from nifi.properties if any are provided
-            identityMappings = Collections.unmodifiableList(IdentityMappingUtil.getIdentityMappings(properties));
-
-            // extract any node identities
-            initialUserIdentities = new HashSet<>();
-            for (Map.Entry<String,String> entry : configurationContext.getProperties().entrySet()) {
-                Matcher matcher = INITIAL_USER_IDENTITY_PATTERN.matcher(entry.getKey());
-                if (matcher.matches() && !StringUtils.isBlank(entry.getValue())) {
-                    initialUserIdentities.add(IdentityMappingUtil.mapIdentity(entry.getValue(), identityMappings));
-                }
-            }
-
-            load();
-
-            // if we've copied the authorizations file to a restore directory synchronize it
-            if (restoreTenantsFile != null) {
-                FileUtils.copyFile(tenantsFile, restoreTenantsFile, false, false, logger);
-            }
-
-            logger.info(String.format("Users/Groups file loaded at %s", new Date().toString()));
-        } catch (IOException | AuthorizerCreationException | JAXBException | IllegalStateException | SAXException e) {
-            throw new AuthorizerCreationException(e);
-        }
-    }
-
-    @Override
-    public Set<User> getUsers() throws AuthorizationAccessException {
-        return userGroupHolder.get().getAllUsers();
-    }
-
-    @Override
-    public synchronized User addUser(User user) throws AuthorizationAccessException {
-        if (user == null) {
-            throw new IllegalArgumentException("User cannot be null");
-        }
-
-        final org.apache.nifi.registry.authorization.file.tenants.generated.User jaxbUser = createJAXBUser(user);
-
-        final UserGroupHolder holder = userGroupHolder.get();
-        final Tenants tenants = holder.getTenants();
-        tenants.getUsers().getUser().add(jaxbUser);
-
-        saveAndRefreshHolder(tenants);
-
-        return userGroupHolder.get().getUsersById().get(user.getIdentifier());
-    }
-
-    @Override
-    public User getUser(String identifier) throws AuthorizationAccessException {
-        if (identifier == null) {
-            return null;
-        }
-
-        final UserGroupHolder holder = userGroupHolder.get();
-        return holder.getUsersById().get(identifier);
-    }
-
-    @Override
-    public synchronized User updateUser(User user) throws AuthorizationAccessException {
-        if (user == null) {
-            throw new IllegalArgumentException("User cannot be null");
-        }
-
-        final UserGroupHolder holder = userGroupHolder.get();
-        final Tenants tenants = holder.getTenants();
-
-        final List<org.apache.nifi.registry.authorization.file.tenants.generated.User> users = tenants.getUsers().getUser();
-
-        // fine the User that needs to be updated
-        org.apache.nifi.registry.authorization.file.tenants.generated.User updateUser = null;
-        for (org.apache.nifi.registry.authorization.file.tenants.generated.User jaxbUser : users) {
-            if (user.getIdentifier().equals(jaxbUser.getIdentifier())) {
-                updateUser = jaxbUser;
-                break;
-            }
-        }
-
-        // if user wasn't found return null, otherwise update the user and save changes
-        if (updateUser == null) {
-            return null;
-        } else {
-            updateUser.setIdentity(user.getIdentity());
-            saveAndRefreshHolder(tenants);
-
-            return userGroupHolder.get().getUsersById().get(user.getIdentifier());
-        }
-    }
-
-    @Override
-    public User getUserByIdentity(String identity) throws AuthorizationAccessException {
-        if (identity == null) {
-            return null;
-        }
-
-        final UserGroupHolder holder = userGroupHolder.get();
-        return holder.getUsersByIdentity().get(identity);
-    }
-
-    @Override
-    public synchronized User deleteUser(User user) throws AuthorizationAccessException {
-        if (user == null) {
-            throw new IllegalArgumentException("User cannot be null");
-        }
-
-        return deleteUser(user.getIdentifier());
-    }
-
-    @Override
-    public synchronized User deleteUser(String userIdentifier) throws AuthorizationAccessException {
-        if (userIdentifier == null) {
-            throw new IllegalArgumentException("User identifier cannot be null");
-        }
-
-        final UserGroupHolder holder = userGroupHolder.get();
-        final User deletedUser = holder.getUsersById().get(userIdentifier);
-        if (deletedUser == null) {
-            return null;
-        }
-
-        // for each group iterate over the user references and remove the user reference if it matches the user being deleted
-        final Tenants tenants = holder.getTenants();
-        for (org.apache.nifi.registry.authorization.file.tenants.generated.Group group : tenants.getGroups().getGroup()) {
-            Iterator<org.apache.nifi.registry.authorization.file.tenants.generated.Group.User> groupUserIter = group.getUser().iterator();
-            while (groupUserIter.hasNext()) {
-                org.apache.nifi.registry.authorization.file.tenants.generated.Group.User groupUser = groupUserIter.next();
-                if (groupUser.getIdentifier().equals(userIdentifier)) {
-                    groupUserIter.remove();
-                    break;
-                }
-            }
-        }
-
-        // remove the actual user
-        Iterator<org.apache.nifi.registry.authorization.file.tenants.generated.User> iter = tenants.getUsers().getUser().iterator();
-        while (iter.hasNext()) {
-            org.apache.nifi.registry.authorization.file.tenants.generated.User jaxbUser = iter.next();
-            if (userIdentifier.equals(jaxbUser.getIdentifier())) {
-                iter.remove();
-                break;
-            }
-        }
-
-        saveAndRefreshHolder(tenants);
-        return deletedUser;
-    }
-
-    @Override
-    public Set<Group> getGroups() throws AuthorizationAccessException {
-        return userGroupHolder.get().getAllGroups();
-    }
-
-    @Override
-    public synchronized Group addGroup(Group group) throws AuthorizationAccessException {
-        if (group == null) {
-            throw new IllegalArgumentException("Group cannot be null");
-        }
-
-        final UserGroupHolder holder = userGroupHolder.get();
-        final Tenants tenants = holder.getTenants();
-
-        // determine that all users in the group exist before doing anything, throw an exception if they don't
-        checkGroupUsers(group, tenants.getUsers().getUser());
-
-        // create a new JAXB Group based on the incoming Group
-        final org.apache.nifi.registry.authorization.file.tenants.generated.Group jaxbGroup =
-                new org.apache.nifi.registry.authorization.file.tenants.generated.Group();
-        jaxbGroup.setIdentifier(group.getIdentifier());
-        jaxbGroup.setName(group.getName());
-
-        // add each user to the group
-        for (String groupUser : group.getUsers()) {
-            org.apache.nifi.registry.authorization.file.tenants.generated.Group.User jaxbGroupUser =
-                    new org.apache.nifi.registry.authorization.file.tenants.generated.Group.User();
-            jaxbGroupUser.setIdentifier(groupUser);
-            jaxbGroup.getUser().add(jaxbGroupUser);
-        }
-
-        tenants.getGroups().getGroup().add(jaxbGroup);
-        saveAndRefreshHolder(tenants);
-
-        return userGroupHolder.get().getGroupsById().get(group.getIdentifier());
-    }
-
-    @Override
-    public Group getGroup(String identifier) throws AuthorizationAccessException {
-        if (identifier == null) {
-            return null;
-        }
-        return userGroupHolder.get().getGroupsById().get(identifier);
-    }
-
-    @Override
-    public UserAndGroups getUserAndGroups(final String identity) throws AuthorizationAccessException {
-        final UserGroupHolder holder = userGroupHolder.get();
-        final User user = holder.getUser(identity);
-        final Set<Group> groups = holder.getGroups(identity);
-
-        return new UserAndGroups() {
-            @Override
-            public User getUser() {
-                return user;
-            }
-
-            @Override
-            public Set<Group> getGroups() {
-                return groups;
-            }
-        };
-    }
-
-    @Override
-    public synchronized Group updateGroup(Group group) throws AuthorizationAccessException {
-        if (group == null) {
-            throw new IllegalArgumentException("Group cannot be null");
-        }
-
-        final UserGroupHolder holder = userGroupHolder.get();
-        final Tenants tenants = holder.getTenants();
-
-        // find the group that needs to be update
-        org.apache.nifi.registry.authorization.file.tenants.generated.Group updateGroup = null;
-        for (org.apache.nifi.registry.authorization.file.tenants.generated.Group jaxbGroup : tenants.getGroups().getGroup()) {
-            if (jaxbGroup.getIdentifier().equals(group.getIdentifier())) {
-                updateGroup = jaxbGroup;
-                break;
-            }
-        }
-
-        // if the group wasn't found return null, otherwise update the group and save changes
-        if (updateGroup == null) {
-            return null;
-        }
-
-        // reset the list of users and add each user to the group
-        updateGroup.getUser().clear();
-        for (String groupUser : group.getUsers()) {
-            org.apache.nifi.registry.authorization.file.tenants.generated.Group.User jaxbGroupUser =
-                    new org.apache.nifi.registry.authorization.file.tenants.generated.Group.User();
-            jaxbGroupUser.setIdentifier(groupUser);
-            updateGroup.getUser().add(jaxbGroupUser);
-        }
-
-        updateGroup.setName(group.getName());
-        saveAndRefreshHolder(tenants);
-
-        return userGroupHolder.get().getGroupsById().get(group.getIdentifier());
-    }
-
-    @Override
-    public synchronized Group deleteGroup(Group group) throws AuthorizationAccessException {
-        if (group == null) {
-            throw new IllegalArgumentException("Group cannot be null");
-        }
-
-        return deleteGroup(group.getIdentifier());
-    }
-
-    @Override
-    public synchronized Group deleteGroup(String groupIdentifier) throws AuthorizationAccessException {
-        if (groupIdentifier == null) {
-            throw new IllegalArgumentException("Group identifier cannot be null");
-        }
-
-        final UserGroupHolder holder = userGroupHolder.get();
-        final Group deletedGroup = holder.getGroupsById().get(groupIdentifier);
-        if (deletedGroup == null) {
-            return null;
-        }
-
-        // now remove the actual group from the top-level list of groups
-        final Tenants tenants = holder.getTenants();
-        Iterator<org.apache.nifi.registry.authorization.file.tenants.generated.Group> iter = tenants.getGroups().getGroup().iterator();
-        while (iter.hasNext()) {
-            org.apache.nifi.registry.authorization.file.tenants.generated.Group jaxbGroup = iter.next();
-            if (groupIdentifier.equals(jaxbGroup.getIdentifier())) {
-                iter.remove();
-                break;
-            }
-        }
-
-        saveAndRefreshHolder(tenants);
-        return deletedGroup;
-    }
-
-    UserGroupHolder getUserGroupHolder() {
-        return userGroupHolder.get();
-    }
-
-    @AuthorizerContext
-    public void setNiFiProperties(NiFiRegistryProperties properties) {
-        this.properties = properties;
-    }
-
-    @Override
-    public synchronized void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
-        final UsersAndGroups usersAndGroups = parseUsersAndGroups(fingerprint);
-        usersAndGroups.getUsers().forEach(user -> addUser(user));
-        usersAndGroups.getGroups().forEach(group -> addGroup(group));
-    }
-
-    @Override
-    public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException {
-        try {
-            // ensure we understand the proposed fingerprint
-            parseUsersAndGroups(proposedFingerprint);
-        } catch (final AuthorizationAccessException e) {
-            throw new UninheritableAuthorizationsException("Unable to parse the proposed fingerprint: " + e);
-        }
-
-        final UserGroupHolder usersAndGroups = userGroupHolder.get();
-
-        // ensure we are in a proper state to inherit the fingerprint
-        if (!usersAndGroups.getAllUsers().isEmpty() || !usersAndGroups.getAllGroups().isEmpty()) {
-            throw new UninheritableAuthorizationsException("Proposed fingerprint is not inheritable because the current users and groups is not empty.");
-        }
-    }
-
-    @Override
-    public String getFingerprint() throws AuthorizationAccessException {
-        final UserGroupHolder usersAndGroups = userGroupHolder.get();
-
-        final List<User> users = new ArrayList<>(usersAndGroups.getAllUsers());
-        Collections.sort(users, Comparator.comparing(User::getIdentifier));
-
-        final List<Group> groups = new ArrayList<>(usersAndGroups.getAllGroups());
-        Collections.sort(groups, Comparator.comparing(Group::getIdentifier));
-
-        XMLStreamWriter writer = null;
-        final StringWriter out = new StringWriter();
-        try {
-            writer = XML_OUTPUT_FACTORY.createXMLStreamWriter(out);
-            writer.writeStartDocument();
-            writer.writeStartElement("tenants");
-
-            for (User user : users) {
-                writeUser(writer, user);
-            }
-            for (Group group : groups) {
-                writeGroup(writer, group);
-            }
-
-            writer.writeEndElement();
-            writer.writeEndDocument();
-            writer.flush();
-        } catch (XMLStreamException e) {
-            throw new AuthorizationAccessException("Unable to generate fingerprint", e);
-        } finally {
-            if (writer != null) {
-                try {
-                    writer.close();
-                } catch (XMLStreamException e) {
-                    // nothing to do here
-                }
-            }
-        }
-
-        return out.toString();
-    }
-
-    private UsersAndGroups parseUsersAndGroups(final String fingerprint) {
-        final List<User> users = new ArrayList<>();
-        final List<Group> groups = new ArrayList<>();
-
-        final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
-        try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
-            final DocumentBuilder docBuilder = DOCUMENT_BUILDER_FACTORY.newDocumentBuilder();
-            final Document document = docBuilder.parse(in);
-            final Element rootElement = document.getDocumentElement();
-
-            // parse all the users and add them to the current user group provider
-            NodeList userNodes = rootElement.getElementsByTagName(USER_ELEMENT);
-            for (int i=0; i < userNodes.getLength(); i++) {
-                Node userNode = userNodes.item(i);
-                users.add(parseUser((Element) userNode));
-            }
-
-            // parse all the groups and add them to the current user group provider
-            NodeList groupNodes = rootElement.getElementsByTagName(GROUP_ELEMENT);
-            for (int i=0; i < groupNodes.getLength(); i++) {
-                Node groupNode = groupNodes.item(i);
-                groups.add(parseGroup((Element) groupNode));
-            }
-        } catch (SAXException | ParserConfigurationException | IOException e) {
-            throw new AuthorizationAccessException("Unable to parse fingerprint", e);
-        }
-
-        return new UsersAndGroups(users, groups);
-    }
-
-    private User parseUser(final Element element) {
-        final User.Builder builder = new User.Builder()
-                .identifier(element.getAttribute(IDENTIFIER_ATTR))
-                .identity(element.getAttribute(IDENTITY_ATTR));
-
-        return builder.build();
-    }
-
-    private Group parseGroup(final Element element) {
-        final Group.Builder builder = new Group.Builder()
-                .identifier(element.getAttribute(IDENTIFIER_ATTR))
-                .name(element.getAttribute(NAME_ATTR));
-
-        NodeList groupUsers = element.getElementsByTagName(GROUP_USER_ELEMENT);
-        for (int i=0; i < groupUsers.getLength(); i++) {
-            Element groupUserNode = (Element) groupUsers.item(i);
-            builder.addUser(groupUserNode.getAttribute(IDENTIFIER_ATTR));
-        }
-
-        return builder.build();
-    }
-
-    private void writeUser(final XMLStreamWriter writer, final User user) throws XMLStreamException {
-        writer.writeStartElement(USER_ELEMENT);
-        writer.writeAttribute(IDENTIFIER_ATTR, user.getIdentifier());
-        writer.writeAttribute(IDENTITY_ATTR, user.getIdentity());
-        writer.writeEndElement();
-    }
-
-    private void writeGroup(final XMLStreamWriter writer, final Group group) throws XMLStreamException {
-        List<String> users = new ArrayList<>(group.getUsers());
-        Collections.sort(users);
-
-        writer.writeStartElement(GROUP_ELEMENT);
-        writer.writeAttribute(IDENTIFIER_ATTR, group.getIdentifier());
-        writer.writeAttribute(NAME_ATTR, group.getName());
-
-        for (String user : users) {
-            writer.writeStartElement(GROUP_USER_ELEMENT);
-            writer.writeAttribute(IDENTIFIER_ATTR, user);
-            writer.writeEndElement();
-        }
-
-        writer.writeEndElement();
-    }
-
-    private org.apache.nifi.registry.authorization.file.tenants.generated.User createJAXBUser(User user) {
-        final org.apache.nifi.registry.authorization.file.tenants.generated.User jaxbUser =
-                new org.apache.nifi.registry.authorization.file.tenants.generated.User();
-        jaxbUser.setIdentifier(user.getIdentifier());
-        jaxbUser.setIdentity(user.getIdentity());
-        return jaxbUser;
-    }
-
-    private Set<org.apache.nifi.registry.authorization.file.tenants.generated.User> checkGroupUsers(
-            final Group group,
-            final List<org.apache.nifi.registry.authorization.file.tenants.generated.User> users) {
-        final Set<org.apache.nifi.registry.authorization.file.tenants.generated.User> jaxbUsers = new HashSet<>();
-        for (String groupUser : group.getUsers()) {
-            boolean found = false;
-            for (org.apache.nifi.registry.authorization.file.tenants.generated.User jaxbUser : users) {
-                if (jaxbUser.getIdentifier().equals(groupUser)) {
-                    jaxbUsers.add(jaxbUser);
-                    found = true;
-                    break;
-                }
-            }
-
-            if (!found) {
-                throw new IllegalStateException("Unable to add group because user " + groupUser + " does not exist");
-            }
-        }
-        return jaxbUsers;
-    }
-
-    /**
-     * Loads the authorizations file and populates the AuthorizationsHolder, only called during start-up.
-     *
-     * @throws JAXBException            Unable to reload the authorized users file
-     * @throws IllegalStateException    Unable to sync file with restore
-     * @throws SAXException             Unable to unmarshall tenants
-     */
-    private synchronized void load() throws JAXBException, IllegalStateException, SAXException {
-        final Tenants tenants = unmarshallTenants();
-        if (tenants.getUsers() == null) {
-            tenants.setUsers(new Users());
-        }
-        if (tenants.getGroups() == null) {
-            tenants.setGroups(new Groups());
-        }
-
-        final UserGroupHolder userGroupHolder = new UserGroupHolder(tenants);
-        final boolean emptyTenants = userGroupHolder.getAllUsers().isEmpty() && userGroupHolder.getAllGroups().isEmpty();
-//        final boolean hasLegacyAuthorizedUsers = (legacyAuthorizedUsersFile != null && !StringUtils.isBlank(legacyAuthorizedUsersFile));
-
-        if (emptyTenants) {
-//            if (hasLegacyAuthorizedUsers) {
-//                logger.info("Loading users from legacy model " + legacyAuthorizedUsersFile + " into new users file.");
-//                convertLegacyAuthorizedUsers(tenants);
-//            }
-
-            populateInitialUsers(tenants);
-
-            // save any changes that were made and repopulate the holder
-            saveAndRefreshHolder(tenants);
-        } else {
-            this.userGroupHolder.set(userGroupHolder);
-        }
-    }
-
-    private void saveTenants(final Tenants tenants) throws JAXBException {
-        final Marshaller marshaller = JAXB_TENANTS_CONTEXT.createMarshaller();
-        marshaller.setSchema(tenantsSchema);
-        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
-        marshaller.marshal(tenants, tenantsFile);
-    }
-
-    private Tenants unmarshallTenants() throws JAXBException {
-        final Unmarshaller unmarshaller = JAXB_TENANTS_CONTEXT.createUnmarshaller();
-        unmarshaller.setSchema(tenantsSchema);
-
-        final JAXBElement<Tenants> element = unmarshaller.unmarshal(new StreamSource(tenantsFile), Tenants.class);
-        return element.getValue();
-    }
-
-    private void populateInitialUsers(final Tenants tenants) {
-        for (String initialUserIdentity : initialUserIdentities) {
-            getOrCreateUser(tenants, initialUserIdentity);
-        }
-    }
-
-    /**
-     * Finds the User with the given identity, or creates a new one and adds it to the Tenants.
-     *
-     * @param tenants the Tenants reference
-     * @param userIdentity the user identity to find or create
-     * @return the User from Tenants with the given identity, or a new instance that was added to Tenants
-     */
-    private org.apache.nifi.registry.authorization.file.tenants.generated.User getOrCreateUser(final Tenants tenants, final String userIdentity) {
-        if (StringUtils.isBlank(userIdentity)) {
-            return null;
-        }
-
-        org.apache.nifi.registry.authorization.file.tenants.generated.User foundUser = null;
-        for (org.apache.nifi.registry.authorization.file.tenants.generated.User user : tenants.getUsers().getUser()) {
-            if (user.getIdentity().equals(userIdentity)) {
-                foundUser = user;
-                break;
-            }
-        }
-
-        if (foundUser == null) {
-            final String userIdentifier = IdentifierUtil.getIdentifier(userIdentity);
-            foundUser = new org.apache.nifi.registry.authorization.file.tenants.generated.User();
-            foundUser.setIdentifier(userIdentifier);
-            foundUser.setIdentity(userIdentity);
-            tenants.getUsers().getUser().add(foundUser);
-        }
-
-        return foundUser;
-    }
-
-    /**
-     * Finds the Group with the given name, or creates a new one and adds it to Tenants.
-     *
-     * @param tenants the Tenants reference
-     * @param groupName the name of the group to look for
-     * @return the Group from Tenants with the given name, or a new instance that was added to Tenants
-     */
-    private org.apache.nifi.registry.authorization.file.tenants.generated.Group getOrCreateGroup(final Tenants tenants, final String groupName) {
-        if (StringUtils.isBlank(groupName)) {
-            return null;
-        }
-
-        org.apache.nifi.registry.authorization.file.tenants.generated.Group foundGroup = null;
-        for (org.apache.nifi.registry.authorization.file.tenants.generated.Group group : tenants.getGroups().getGroup()) {
-            if (group.getName().equals(groupName)) {
-                foundGroup = group;
-                break;
-            }
-        }
-
-        if (foundGroup == null) {
-            final String newGroupIdentifier = IdentifierUtil.getIdentifier(groupName);
-            foundGroup = new org.apache.nifi.registry.authorization.file.tenants.generated.Group();
-            foundGroup.setIdentifier(newGroupIdentifier);
-            foundGroup.setName(groupName);
-            tenants.getGroups().getGroup().add(foundGroup);
-        }
-
-        return foundGroup;
-    }
-
-    /**
-     * Saves the Authorizations instance by marshalling to a file, then re-populates the
-     * in-memory data structures and sets the new holder.
-     *
-     * Synchronized to ensure only one thread writes the file at a time.
-     *
-     * @param tenants the tenants to save and populate from
-     * @throws AuthorizationAccessException if an error occurs saving the authorizations
-     */
-    private synchronized void saveAndRefreshHolder(final Tenants tenants) throws AuthorizationAccessException {
-        try {
-            saveTenants(tenants);
-
-            this.userGroupHolder.set(new UserGroupHolder(tenants));
-        } catch (JAXBException e) {
-            throw new AuthorizationAccessException("Unable to save Authorizations", e);
-        }
-    }
-
-    @Override
-    public void preDestruction() throws AuthorizerDestructionException {
-    }
-
-    private static class UsersAndGroups {
-        final List<User> users;
-        final List<Group> groups;
-
-        public UsersAndGroups(List<User> users, List<Group> groups) {
-            this.users = users;
-            this.groups = groups;
-        }
-
-        public List<User> getUsers() {
-            return users;
-        }
-
-        public List<Group> getGroups() {
-            return groups;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/IdentifierUtil.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/IdentifierUtil.java b/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/IdentifierUtil.java
deleted file mode 100644
index b698cd0..0000000
--- a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/IdentifierUtil.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.file;
-
-import org.apache.commons.lang3.StringUtils;
-
-import java.nio.charset.StandardCharsets;
-import java.util.UUID;
-
-public final class IdentifierUtil {
-
-    static String getIdentifier(final String seed) {
-        if (StringUtils.isBlank(seed)) {
-            return null;
-        }
-
-        return UUID.nameUUIDFromBytes(seed.getBytes(StandardCharsets.UTF_8)).toString();
-    }
-
-    private IdentifierUtil() {}
-}


[16/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizerFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizerFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizerFactory.java
deleted file mode 100644
index cf979dd..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizerFactory.java
+++ /dev/null
@@ -1,820 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.authorization.annotation.AuthorizerContext;
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerDestructionException;
-import org.apache.nifi.registry.authorization.exception.UninheritableAuthorizationsException;
-import org.apache.nifi.registry.authorization.generated.Authorizers;
-import org.apache.nifi.registry.authorization.generated.Prop;
-import org.apache.nifi.registry.extension.ExtensionManager;
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.provider.StandardProviderFactory;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.xml.sax.SAXException;
-
-import javax.xml.XMLConstants;
-import javax.xml.bind.JAXBContext;
-import javax.xml.bind.JAXBElement;
-import javax.xml.bind.JAXBException;
-import javax.xml.bind.Unmarshaller;
-import javax.xml.transform.stream.StreamSource;
-import javax.xml.validation.Schema;
-import javax.xml.validation.SchemaFactory;
-import java.io.File;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * This implementation of AuthorizerFactory in NiFi Registry is based on a combination of
- * NiFi's AuthorizerFactory and AuthorizerFactoryBean.
- */
-public class StandardAuthorizerFactory implements AuthorizerFactory, UserGroupProviderLookup, AccessPolicyProviderLookup, AuthorizerLookup{
-
-    private static final Logger logger = LoggerFactory.getLogger(StandardProviderFactory.class);
-
-    private static final String AUTHORIZERS_XSD = "/authorizers.xsd";
-    private static final String JAXB_GENERATED_PATH = "org.apache.nifi.registry.authorization.generated";
-    private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext();
-
-    /**
-     * Load the JAXBContext.
-     */
-    private static JAXBContext initializeJaxbContext() {
-        try {
-            return JAXBContext.newInstance(JAXB_GENERATED_PATH, StandardAuthorizerFactory.class.getClassLoader());
-        } catch (JAXBException e) {
-            throw new RuntimeException("Unable to create JAXBContext.", e);
-        }
-    }
-
-    private final NiFiRegistryProperties properties;
-    private final ExtensionManager extensionManager;
-
-    private Authorizer authorizer;
-    private final Map<String, UserGroupProvider> userGroupProviders = new HashMap<>();
-    private final Map<String, AccessPolicyProvider> accessPolicyProviders = new HashMap<>();
-    private final Map<String, Authorizer> authorizers = new HashMap<>();
-
-    public StandardAuthorizerFactory(final NiFiRegistryProperties properties, final ExtensionManager extensionManager) {
-        this.properties = properties;
-        this.extensionManager = extensionManager;
-
-        if (this.properties == null) {
-            throw new IllegalStateException("NiFiRegistryProperties cannot be null");
-        }
-
-        if (this.extensionManager == null) {
-            throw new IllegalStateException("ExtensionManager cannot be null");
-        }
-    }
-
-    /***** UserGroupProviderLookup *****/
-
-    @Override
-    public UserGroupProvider getUserGroupProvider(String identifier) {
-        return userGroupProviders.get(identifier);
-    }
-
-    /***** AccessPolicyProviderLookup *****/
-
-    @Override
-    public AccessPolicyProvider getAccessPolicyProvider(String identifier) {
-        return accessPolicyProviders.get(identifier);
-    }
-
-
-    /***** AuthorizerLookup *****/
-
-    @Override
-    public Authorizer getAuthorizer(String identifier) {
-        return authorizers.get(identifier);
-    }
-
-
-    /***** AuthorizerFactory *****/
-
-    @Override
-    public void initialize() throws AuthorizerFactoryException {
-//        if (authorizerHolder.get() == null) {
-//            final File authorizersConfigFile = properties.getAuthorizersConfigurationFile();
-//            if (authorizersConfigFile.exists()) {
-//                try {
-//                    // find the schema
-//                    final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-//                    final Schema schema = schemaFactory.newSchema(StandardProviderFactory.class.getResource(AUTHORIZERS_XSD));
-//
-//                    // attempt to unmarshal
-//                    final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
-//                    unmarshaller.setSchema(schema);
-//
-//                    // set the holder for later use
-//                    final JAXBElement<Authorizers> element = unmarshaller.unmarshal(new StreamSource(authorizersConfigFile), Authorizers.class);
-//                    authorizerHolder.set(element.getValue());
-//                } catch (SAXException | JAXBException e) {
-//                    throw new AuthorizerFactoryException("Unable to load the authorizer configuration file at: " + authorizersConfigFile.getAbsolutePath(), e);
-//                }
-//            } else {
-//                throw new AuthorizerFactoryException("Unable to find the providers configuration file at " + authorizersConfigFile.getAbsolutePath());
-//            }
-//        }
-    }
-
-    @Override
-    public Authorizer getAuthorizer() throws AuthorizerFactoryException {
-        if (authorizer == null) {
-            if (properties.getSslPort() == null) {
-                // use a default authorizer... only allowable when running not securely
-                authorizer = createDefaultAuthorizer();
-            } else {
-                // look up the authorizer to use
-                final String authorizerIdentifier = properties.getProperty(NiFiRegistryProperties.SECURITY_AUTHORIZER);
-
-                // ensure the authorizer class name was specified
-                if (StringUtils.isBlank(authorizerIdentifier)) {
-                    throw new AuthorizerFactoryException("When running securely, the authorizer identifier must be specified in the nifi properties file.");
-                } else {
-
-                    try {
-                        final Authorizers authorizerConfiguration = loadAuthorizersConfiguration();
-
-                        // create each user group provider
-                        for (final org.apache.nifi.registry.authorization.generated.UserGroupProvider userGroupProvider : authorizerConfiguration.getUserGroupProvider()) {
-                            userGroupProviders.put(userGroupProvider.getIdentifier(), createUserGroupProvider(userGroupProvider.getIdentifier(), userGroupProvider.getClazz()));
-                        }
-
-                        // configure each user group provider
-                        for (final org.apache.nifi.registry.authorization.generated.UserGroupProvider provider : authorizerConfiguration.getUserGroupProvider()) {
-                            final UserGroupProvider instance = userGroupProviders.get(provider.getIdentifier());
-                            instance.onConfigured(loadAuthorizerConfiguration(provider.getIdentifier(), provider.getProperty()));
-                        }
-
-                        // create each access policy provider
-                        for (final org.apache.nifi.registry.authorization.generated.AccessPolicyProvider accessPolicyProvider : authorizerConfiguration.getAccessPolicyProvider()) {
-                            accessPolicyProviders.put(accessPolicyProvider.getIdentifier(), createAccessPolicyProvider(accessPolicyProvider.getIdentifier(), accessPolicyProvider.getClazz()));
-                        }
-
-                        // configure each access policy provider
-                        for (final org.apache.nifi.registry.authorization.generated.AccessPolicyProvider provider : authorizerConfiguration.getAccessPolicyProvider()) {
-                            final AccessPolicyProvider instance = accessPolicyProviders.get(provider.getIdentifier());
-                            instance.onConfigured(loadAuthorizerConfiguration(provider.getIdentifier(), provider.getProperty()));
-                        }
-
-                        // create each authorizer
-                        for (final org.apache.nifi.registry.authorization.generated.Authorizer authorizer : authorizerConfiguration.getAuthorizer()) {
-                            authorizers.put(authorizer.getIdentifier(), createAuthorizer(authorizer.getIdentifier(), authorizer.getClazz(), authorizer.getClasspath()));
-                        }
-
-                        // configure each authorizer
-                        for (final org.apache.nifi.registry.authorization.generated.Authorizer provider : authorizerConfiguration.getAuthorizer()) {
-                            final Authorizer instance = authorizers.get(provider.getIdentifier());
-                            instance.onConfigured(loadAuthorizerConfiguration(provider.getIdentifier(), provider.getProperty()));
-                        }
-
-                        // get the authorizer instance
-                        authorizer = getAuthorizer(authorizerIdentifier);
-
-                        // ensure it was found
-                        if (authorizer == null) {
-                            throw new AuthorizerFactoryException(String.format("The specified authorizer '%s' could not be found.", authorizerIdentifier));
-                        }
-                    } catch (Exception e) {
-                        throw new AuthorizerFactoryException("Failed to construct Authorizer.", e);
-                    }
-                }
-            }
-        }
-        return authorizer;
-    }
-
-    private Authorizers loadAuthorizersConfiguration() throws Exception {
-        final File authorizersConfigurationFile = properties.getAuthorizersConfigurationFile();
-
-        // load the authorizers from the specified file
-        if (authorizersConfigurationFile.exists()) {
-            try {
-                // find the schema
-                final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
-                final Schema schema = schemaFactory.newSchema(Authorizers.class.getResource(AUTHORIZERS_XSD));
-
-                // attempt to unmarshal
-                final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller();
-                unmarshaller.setSchema(schema);
-                final JAXBElement<Authorizers> element = unmarshaller.unmarshal(new StreamSource(authorizersConfigurationFile), Authorizers.class);
-                return element.getValue();
-            } catch (SAXException | JAXBException e) {
-                throw new Exception("Unable to load the authorizer configuration file at: " + authorizersConfigurationFile.getAbsolutePath(), e);
-            }
-        } else {
-            throw new Exception("Unable to find the authorizer configuration file at " + authorizersConfigurationFile.getAbsolutePath());
-        }
-    }
-
-    private AuthorizerConfigurationContext loadAuthorizerConfiguration(final String identifier, final List<Prop> properties) {
-        final Map<String, String> authorizerProperties = new HashMap<>();
-
-        for (final Prop property : properties) {
-            authorizerProperties.put(property.getName(), property.getValue());
-        }
-        return new StandardAuthorizerConfigurationContext(identifier, authorizerProperties);
-    }
-
-    private UserGroupProvider createUserGroupProvider(final String identifier, final String userGroupProviderClassName) throws Exception {
-
-        final UserGroupProvider instance;
-
-        final ClassLoader classLoader = extensionManager.getExtensionClassLoader(userGroupProviderClassName);
-        if (classLoader == null) {
-            throw new IllegalStateException("Extension not found in any of the configured class loaders: " + userGroupProviderClassName);
-        }
-
-        // attempt to load the class
-        Class<?> rawUserGroupProviderClass = Class.forName(userGroupProviderClassName, true, classLoader);
-        Class<? extends UserGroupProvider> userGroupProviderClass = rawUserGroupProviderClass.asSubclass(UserGroupProvider.class);
-
-        // otherwise create a new instance
-        Constructor constructor = userGroupProviderClass.getConstructor();
-        instance = (UserGroupProvider) constructor.newInstance();
-
-        // method injection
-        performMethodInjection(instance, userGroupProviderClass);
-
-        // field injection
-        performFieldInjection(instance, userGroupProviderClass);
-
-        // call post construction lifecycle event
-        instance.initialize(new StandardAuthorizerInitializationContext(identifier, this, this, this));
-
-        return instance;
-    }
-
-    private AccessPolicyProvider createAccessPolicyProvider(final String identifier, final String accessPolicyProviderClassName) throws Exception {
-        final AccessPolicyProvider instance;
-
-        final ClassLoader classLoader = extensionManager.getExtensionClassLoader(accessPolicyProviderClassName);
-        if (classLoader == null) {
-            throw new IllegalStateException("Extension not found in any of the configured class loaders: " + accessPolicyProviderClassName);
-        }
-
-        // attempt to load the class
-        Class<?> rawAccessPolicyProviderClass = Class.forName(accessPolicyProviderClassName, true, classLoader);
-        Class<? extends AccessPolicyProvider> accessPolicyClass = rawAccessPolicyProviderClass.asSubclass(AccessPolicyProvider.class);
-
-        // otherwise create a new instance
-        Constructor constructor = accessPolicyClass.getConstructor();
-        instance = (AccessPolicyProvider) constructor.newInstance();
-
-        // method injection
-        performMethodInjection(instance, accessPolicyClass);
-
-        // field injection
-        performFieldInjection(instance, accessPolicyClass);
-
-        // call post construction lifecycle event
-        instance.initialize(new StandardAuthorizerInitializationContext(identifier, this, this, this));
-
-        return instance;
-    }
-
-    private Authorizer createAuthorizer(final String identifier, final String authorizerClassName, final String classpathResources) throws Exception {
-        final Authorizer instance;
-
-        final ClassLoader classLoader = extensionManager.getExtensionClassLoader(authorizerClassName);
-        if (classLoader == null) {
-            throw new IllegalStateException("Extension not found in any of the configured class loaders: " + authorizerClassName);
-        }
-
-        // attempt to load the class
-        Class<?> rawAuthorizerClass = Class.forName(authorizerClassName, true, classLoader);
-        Class<? extends Authorizer> authorizerClass = rawAuthorizerClass.asSubclass(Authorizer.class);
-
-        // otherwise create a new instance
-        Constructor constructor = authorizerClass.getConstructor();
-        instance = (Authorizer) constructor.newInstance();
-
-        // method injection
-        performMethodInjection(instance, authorizerClass);
-
-        // field injection
-        performFieldInjection(instance, authorizerClass);
-
-        // call post construction lifecycle event
-        instance.initialize(new StandardAuthorizerInitializationContext(identifier, this, this, this));
-
-        // TODO, implement and test loading additional resources from the classpath for custom authorizer impls.
-//        if (StringUtils.isNotEmpty(classpathResources)) {
-//            URL[] urls = ClassLoaderUtils.getURLsForClasspath(classpathResources, null, true);
-//            authorizerClassLoader = new URLClassLoader(urls, authorizerClassLoader);
-//        }
-
-        return installIntegrityChecks(instance);
-    }
-
-        private void performMethodInjection(final Object instance, final Class authorizerClass) throws IllegalAccessException, IllegalArgumentException, InvocationTargetException {
-        for (final Method method : authorizerClass.getMethods()) {
-            if (method.isAnnotationPresent(AuthorizerContext.class)) {
-                // make the method accessible
-                final boolean isAccessible = method.isAccessible();
-                method.setAccessible(true);
-
-                try {
-                    final Class<?>[] argumentTypes = method.getParameterTypes();
-
-                    // look for setters (single argument)
-                    if (argumentTypes.length == 1) {
-                        final Class<?> argumentType = argumentTypes[0];
-
-                        // look for well known types
-                        if (NiFiRegistryProperties.class.isAssignableFrom(argumentType)) {
-                            // nifi properties injection
-                            method.invoke(instance, properties);
-                        }
-                    }
-                } finally {
-                    method.setAccessible(isAccessible);
-                }
-            }
-        }
-
-        final Class parentClass = authorizerClass.getSuperclass();
-        if (parentClass != null && Authorizer.class.isAssignableFrom(parentClass)) {
-            performMethodInjection(instance, parentClass);
-        }
-    }
-
-    private void performFieldInjection(final Object instance, final Class authorizerClass) throws IllegalArgumentException, IllegalAccessException {
-        for (final Field field : authorizerClass.getDeclaredFields()) {
-            if (field.isAnnotationPresent(AuthorizerContext.class)) {
-                // make the method accessible
-                final boolean isAccessible = field.isAccessible();
-                field.setAccessible(true);
-
-                try {
-                    // get the type
-                    final Class<?> fieldType = field.getType();
-
-                    // only consider this field if it isn't set yet
-                    if (field.get(instance) == null) {
-                        // look for well known types
-                        if (NiFiRegistryProperties.class.isAssignableFrom(fieldType)) {
-                            // nifi properties injection
-                            field.set(instance, properties);
-                        }
-                    }
-
-                } finally {
-                    field.setAccessible(isAccessible);
-                }
-            }
-        }
-
-        final Class parentClass = authorizerClass.getSuperclass();
-        if (parentClass != null && Authorizer.class.isAssignableFrom(parentClass)) {
-            performFieldInjection(instance, parentClass);
-        }
-    }
-
-
-    /**
-     * @return a default Authorizer to use when running unsecurely with no authorizer configured
-     */
-    private Authorizer createDefaultAuthorizer() {
-        return new Authorizer() {
-            @Override
-            public AuthorizationResult authorize(final AuthorizationRequest request) throws AuthorizationAccessException {
-                return AuthorizationResult.approved();
-            }
-
-            @Override
-            public void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
-            }
-
-            @Override
-            public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-            }
-
-            @Override
-            public void preDestruction() throws AuthorizerDestructionException {
-            }
-        };
-    }
-
-    public static Authorizer installIntegrityChecks(final Authorizer baseAuthorizer) {
-        if (baseAuthorizer instanceof ManagedAuthorizer) {
-            final ManagedAuthorizer baseManagedAuthorizer = (ManagedAuthorizer) baseAuthorizer;
-            return new ManagedAuthorizer() {
-                @Override
-                public String getFingerprint() throws AuthorizationAccessException {
-                    return baseManagedAuthorizer.getFingerprint();
-                }
-
-                @Override
-                public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
-                    baseManagedAuthorizer.inheritFingerprint(fingerprint);
-                }
-
-                @Override
-                public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
-                    baseManagedAuthorizer.checkInheritability(proposedFingerprint);
-                }
-
-                @Override
-                public AccessPolicyProvider getAccessPolicyProvider() {
-                    final AccessPolicyProvider baseAccessPolicyProvider = baseManagedAuthorizer.getAccessPolicyProvider();
-                    if (baseAccessPolicyProvider instanceof ConfigurableAccessPolicyProvider) {
-                        final ConfigurableAccessPolicyProvider baseConfigurableAccessPolicyProvider = (ConfigurableAccessPolicyProvider) baseAccessPolicyProvider;
-                        return new ConfigurableAccessPolicyProvider() {
-                            @Override
-                            public String getFingerprint() throws AuthorizationAccessException {
-                                return baseConfigurableAccessPolicyProvider.getFingerprint();
-                            }
-
-                            @Override
-                            public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
-                                baseConfigurableAccessPolicyProvider.inheritFingerprint(fingerprint);
-                            }
-
-                            @Override
-                            public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
-                                baseConfigurableAccessPolicyProvider.checkInheritability(proposedFingerprint);
-                            }
-
-                            @Override
-                            public AccessPolicy addAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
-                                if (policyExists(baseConfigurableAccessPolicyProvider, accessPolicy)) {
-                                    throw new IllegalStateException(String.format("Found multiple policies for '%s' with '%s'.", accessPolicy.getResource(), accessPolicy.getAction()));
-                                }
-                                return baseConfigurableAccessPolicyProvider.addAccessPolicy(accessPolicy);
-                            }
-
-                            @Override
-                            public boolean isConfigurable(AccessPolicy accessPolicy) {
-                                return baseConfigurableAccessPolicyProvider.isConfigurable(accessPolicy);
-                            }
-
-                            @Override
-                            public AccessPolicy updateAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
-                                if (!baseConfigurableAccessPolicyProvider.isConfigurable(accessPolicy)) {
-                                    throw new IllegalArgumentException("The specified access policy is not support modification.");
-                                }
-                                return baseConfigurableAccessPolicyProvider.updateAccessPolicy(accessPolicy);
-                            }
-
-                            @Override
-                            public AccessPolicy deleteAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
-                                if (!baseConfigurableAccessPolicyProvider.isConfigurable(accessPolicy)) {
-                                    throw new IllegalArgumentException("The specified access policy is not support modification.");
-                                }
-                                return baseConfigurableAccessPolicyProvider.deleteAccessPolicy(accessPolicy);
-                            }
-
-                            @Override
-                            public AccessPolicy deleteAccessPolicy(String accessPolicyIdentifier) throws AuthorizationAccessException {
-                                if (!baseConfigurableAccessPolicyProvider.isConfigurable(baseConfigurableAccessPolicyProvider.getAccessPolicy(accessPolicyIdentifier))) {
-                                    throw new IllegalArgumentException("The specified access policy is not support modification.");
-                                }
-                                return baseConfigurableAccessPolicyProvider.deleteAccessPolicy(accessPolicyIdentifier);
-                            }
-
-                            @Override
-                            public Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException {
-                                return baseConfigurableAccessPolicyProvider.getAccessPolicies();
-                            }
-
-                            @Override
-                            public AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException {
-                                return baseConfigurableAccessPolicyProvider.getAccessPolicy(identifier);
-                            }
-
-                            @Override
-                            public AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) throws AuthorizationAccessException {
-                                return baseConfigurableAccessPolicyProvider.getAccessPolicy(resourceIdentifier, action);
-                            }
-
-                            @Override
-                            public UserGroupProvider getUserGroupProvider() {
-                                final UserGroupProvider baseUserGroupProvider = baseConfigurableAccessPolicyProvider.getUserGroupProvider();
-                                if (baseUserGroupProvider instanceof ConfigurableUserGroupProvider) {
-                                    final ConfigurableUserGroupProvider baseConfigurableUserGroupProvider = (ConfigurableUserGroupProvider) baseUserGroupProvider;
-                                    return new ConfigurableUserGroupProvider() {
-                                        @Override
-                                        public String getFingerprint() throws AuthorizationAccessException {
-                                            return baseConfigurableUserGroupProvider.getFingerprint();
-                                        }
-
-                                        @Override
-                                        public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
-                                            baseConfigurableUserGroupProvider.inheritFingerprint(fingerprint);
-                                        }
-
-                                        @Override
-                                        public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
-                                            baseConfigurableUserGroupProvider.checkInheritability(proposedFingerprint);
-                                        }
-
-                                        @Override
-                                        public User addUser(User user) throws AuthorizationAccessException {
-                                            if (tenantExists(baseConfigurableUserGroupProvider, user.getIdentifier(), user.getIdentity())) {
-                                                throw new IllegalStateException(String.format("User/user group already exists with the identity '%s'.", user.getIdentity()));
-                                            }
-                                            return baseConfigurableUserGroupProvider.addUser(user);
-                                        }
-
-                                        @Override
-                                        public boolean isConfigurable(User user) {
-                                            return baseConfigurableUserGroupProvider.isConfigurable(user);
-                                        }
-
-                                        @Override
-                                        public User updateUser(User user) throws AuthorizationAccessException {
-                                            if (tenantExists(baseConfigurableUserGroupProvider, user.getIdentifier(), user.getIdentity())) {
-                                                throw new IllegalStateException(String.format("User/user group already exists with the identity '%s'.", user.getIdentity()));
-                                            }
-                                            if (!baseConfigurableUserGroupProvider.isConfigurable(user)) {
-                                                throw new IllegalArgumentException("The specified user does not support modification.");
-                                            }
-                                            return baseConfigurableUserGroupProvider.updateUser(user);
-                                        }
-
-                                        @Override
-                                        public User deleteUser(User user) throws AuthorizationAccessException {
-                                            if (!baseConfigurableUserGroupProvider.isConfigurable(user)) {
-                                                throw new IllegalArgumentException("The specified user does not support modification.");
-                                            }
-                                            return baseConfigurableUserGroupProvider.deleteUser(user);
-                                        }
-
-                                        @Override
-                                        public User deleteUser(String userIdentifier) throws AuthorizationAccessException {
-                                            if (!baseConfigurableUserGroupProvider.isConfigurable(baseConfigurableUserGroupProvider.getUser(userIdentifier))) {
-                                                throw new IllegalArgumentException("The specified user does not support modification.");
-                                            }
-                                            return baseConfigurableUserGroupProvider.deleteUser(userIdentifier);
-                                        }
-
-                                        @Override
-                                        public Group addGroup(Group group) throws AuthorizationAccessException {
-                                            if (tenantExists(baseConfigurableUserGroupProvider, group.getIdentifier(), group.getName())) {
-                                                throw new IllegalStateException(String.format("User/user group already exists with the identity '%s'.", group.getName()));
-                                            }
-                                            return baseConfigurableUserGroupProvider.addGroup(group);
-                                        }
-
-                                        @Override
-                                        public boolean isConfigurable(Group group) {
-                                            return baseConfigurableUserGroupProvider.isConfigurable(group);
-                                        }
-
-                                        @Override
-                                        public Group updateGroup(Group group) throws AuthorizationAccessException {
-                                            if (tenantExists(baseConfigurableUserGroupProvider, group.getIdentifier(), group.getName())) {
-                                                throw new IllegalStateException(String.format("User/user group already exists with the identity '%s'.", group.getName()));
-                                            }
-                                            if (!baseConfigurableUserGroupProvider.isConfigurable(group)) {
-                                                throw new IllegalArgumentException("The specified group does not support modification.");
-                                            }
-                                            return baseConfigurableUserGroupProvider.updateGroup(group);
-                                        }
-
-                                        @Override
-                                        public Group deleteGroup(Group group) throws AuthorizationAccessException {
-                                            if (!baseConfigurableUserGroupProvider.isConfigurable(group)) {
-                                                throw new IllegalArgumentException("The specified group does not support modification.");
-                                            }
-                                            return baseConfigurableUserGroupProvider.deleteGroup(group);
-                                        }
-
-                                        @Override
-                                        public Group deleteGroup(String groupId) throws AuthorizationAccessException {
-                                            if (!baseConfigurableUserGroupProvider.isConfigurable(baseConfigurableUserGroupProvider.getGroup(groupId))) {
-                                                throw new IllegalArgumentException("The specified group does not support modification.");
-                                            }
-                                            return baseConfigurableUserGroupProvider.deleteGroup(groupId);
-                                        }
-
-                                        @Override
-                                        public Set<User> getUsers() throws AuthorizationAccessException {
-                                            return baseConfigurableUserGroupProvider.getUsers();
-                                        }
-
-                                        @Override
-                                        public User getUser(String identifier) throws AuthorizationAccessException {
-                                            return baseConfigurableUserGroupProvider.getUser(identifier);
-                                        }
-
-                                        @Override
-                                        public User getUserByIdentity(String identity) throws AuthorizationAccessException {
-                                            return baseConfigurableUserGroupProvider.getUserByIdentity(identity);
-                                        }
-
-                                        @Override
-                                        public Set<Group> getGroups() throws AuthorizationAccessException {
-                                            return baseConfigurableUserGroupProvider.getGroups();
-                                        }
-
-                                        @Override
-                                        public Group getGroup(String identifier) throws AuthorizationAccessException {
-                                            return baseConfigurableUserGroupProvider.getGroup(identifier);
-                                        }
-
-                                        @Override
-                                        public UserAndGroups getUserAndGroups(String identity) throws AuthorizationAccessException {
-                                            return baseConfigurableUserGroupProvider.getUserAndGroups(identity);
-                                        }
-
-                                        @Override
-                                        public void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
-                                            baseConfigurableUserGroupProvider.initialize(initializationContext);
-                                        }
-
-                                        @Override
-                                        public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-                                            baseConfigurableUserGroupProvider.onConfigured(configurationContext);
-                                        }
-
-                                        @Override
-                                        public void preDestruction() throws AuthorizerDestructionException {
-                                            baseConfigurableUserGroupProvider.preDestruction();
-                                        }
-                                    };
-                                } else {
-                                    return baseUserGroupProvider;
-                                }
-                            }
-
-                            @Override
-                            public void initialize(AccessPolicyProviderInitializationContext initializationContext) throws AuthorizerCreationException {
-                                baseConfigurableAccessPolicyProvider.initialize(initializationContext);
-                            }
-
-                            @Override
-                            public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-                                baseConfigurableAccessPolicyProvider.onConfigured(configurationContext);
-                            }
-
-                            @Override
-                            public void preDestruction() throws AuthorizerDestructionException {
-                                baseConfigurableAccessPolicyProvider.preDestruction();
-                            }
-                        };
-                    } else {
-                        return baseAccessPolicyProvider;
-                    }
-                }
-
-                @Override
-                public AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException {
-                    final AuthorizationResult result = baseAuthorizer.authorize(request);
-
-                    // audit the authorization request
-                    audit(baseAuthorizer, request, result);
-
-                    return result;
-                }
-
-                @Override
-                public void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
-                    baseManagedAuthorizer.initialize(initializationContext);
-                }
-
-                @Override
-                public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-                    baseManagedAuthorizer.onConfigured(configurationContext);
-
-                    final AccessPolicyProvider accessPolicyProvider = baseManagedAuthorizer.getAccessPolicyProvider();
-                    final UserGroupProvider userGroupProvider = accessPolicyProvider.getUserGroupProvider();
-
-                    // ensure that only one policy per resource-action exists
-                    for (AccessPolicy accessPolicy : accessPolicyProvider.getAccessPolicies()) {
-                        if (policyExists(accessPolicyProvider, accessPolicy)) {
-                            throw new AuthorizerCreationException(String.format("Found multiple policies for '%s' with '%s'.", accessPolicy.getResource(), accessPolicy.getAction()));
-                        }
-                    }
-
-                    // ensure that only one group exists per identity
-                    for (User user : userGroupProvider.getUsers()) {
-                        if (tenantExists(userGroupProvider, user.getIdentifier(), user.getIdentity())) {
-                            throw new AuthorizerCreationException(String.format("Found multiple users/user groups with identity '%s'.", user.getIdentity()));
-                        }
-                    }
-
-                    // ensure that only one group exists per identity
-                    for (Group group : userGroupProvider.getGroups()) {
-                        if (tenantExists(userGroupProvider, group.getIdentifier(), group.getName())) {
-                            throw new AuthorizerCreationException(String.format("Found multiple users/user groups with name '%s'.", group.getName()));
-                        }
-                    }
-                }
-
-                @Override
-                public void preDestruction() throws AuthorizerDestructionException {
-                    baseManagedAuthorizer.preDestruction();
-                }
-            };
-        } else {
-            return new Authorizer() {
-                @Override
-                public AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException {
-                    final AuthorizationResult result = baseAuthorizer.authorize(request);
-
-                    // audit the authorization request
-                    audit(baseAuthorizer, request, result);
-
-                    return result;
-                }
-
-                @Override
-                public void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
-                    baseAuthorizer.initialize(initializationContext);
-                }
-
-                @Override
-                public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-                    baseAuthorizer.onConfigured(configurationContext);
-                }
-
-                @Override
-                public void preDestruction() throws AuthorizerDestructionException {
-                    baseAuthorizer.preDestruction();
-                }
-            };
-        }
-    }
-
-    private static void audit(final Authorizer authorizer, final AuthorizationRequest request, final AuthorizationResult result) {
-        // audit when...
-        // 1 - the authorizer supports auditing
-        // 2 - the request is an access attempt
-        // 3 - the result is either approved/denied, when resource is not found a subsequent request may be following with the parent resource
-        if (authorizer instanceof AuthorizationAuditor && request.isAccessAttempt() && !AuthorizationResult.Result.ResourceNotFound.equals(result.getResult())) {
-            ((AuthorizationAuditor) authorizer).auditAccessAttempt(request, result);
-        }
-    }
-
-    /**
-     * Checks if another policy exists with the same resource and action as the given policy.
-     *
-     * @param checkAccessPolicy an access policy being checked
-     * @return true if another access policy exists with the same resource and action, false otherwise
-     */
-    private static boolean policyExists(final AccessPolicyProvider accessPolicyProvider, final AccessPolicy checkAccessPolicy) {
-        for (AccessPolicy accessPolicy : accessPolicyProvider.getAccessPolicies()) {
-            if (!accessPolicy.getIdentifier().equals(checkAccessPolicy.getIdentifier())
-                    && accessPolicy.getResource().equals(checkAccessPolicy.getResource())
-                    && accessPolicy.getAction().equals(checkAccessPolicy.getAction())) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    /**
-     * Checks if another user exists with the same identity.
-     *
-     * @param identifier identity of the user
-     * @param identity identity of the user
-     * @return true if another user exists with the same identity, false otherwise
-     */
-    private static boolean tenantExists(final UserGroupProvider userGroupProvider, final String identifier, final String identity) {
-        for (User user : userGroupProvider.getUsers()) {
-            if (!user.getIdentifier().equals(identifier)
-                    && user.getIdentity().equals(identity)) {
-                return true;
-            }
-        }
-
-        for (Group group : userGroupProvider.getGroups()) {
-            if (!group.getIdentifier().equals(identifier)
-                    && group.getName().equals(identity)) {
-                return true;
-            }
-        }
-
-        return false;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardManagedAuthorizer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardManagedAuthorizer.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardManagedAuthorizer.java
deleted file mode 100644
index 8cd4fea..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardManagedAuthorizer.java
+++ /dev/null
@@ -1,264 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.authorization.exception.UninheritableAuthorizationsException;
-import org.apache.nifi.registry.util.PropertyValue;
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerDestructionException;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.stream.XMLOutputFactory;
-import javax.xml.stream.XMLStreamException;
-import javax.xml.stream.XMLStreamWriter;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.StringWriter;
-import java.nio.charset.StandardCharsets;
-import java.util.Set;
-
-public class StandardManagedAuthorizer implements ManagedAuthorizer {
-
-    private static final DocumentBuilderFactory DOCUMENT_BUILDER_FACTORY = DocumentBuilderFactory.newInstance();
-    private static final XMLOutputFactory XML_OUTPUT_FACTORY = XMLOutputFactory.newInstance();
-
-    private static final String USER_GROUP_PROVIDER_ELEMENT = "userGroupProvider";
-    private static final String ACCESS_POLICY_PROVIDER_ELEMENT = "accessPolicyProvider";
-
-    private AccessPolicyProviderLookup accessPolicyProviderLookup;
-    private AccessPolicyProvider accessPolicyProvider;
-    private UserGroupProvider userGroupProvider;
-
-    public StandardManagedAuthorizer() {}
-
-    // exposed for testing to inject mocks
-    public StandardManagedAuthorizer(AccessPolicyProvider accessPolicyProvider, UserGroupProvider userGroupProvider) {
-        this.accessPolicyProvider = accessPolicyProvider;
-        this.userGroupProvider = userGroupProvider;
-    }
-
-    @Override
-    public void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
-        accessPolicyProviderLookup = initializationContext.getAccessPolicyProviderLookup();
-    }
-
-    @Override
-    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-        final PropertyValue accessPolicyProviderKey = configurationContext.getProperty("Access Policy Provider");
-        if (!accessPolicyProviderKey.isSet()) {
-            throw new AuthorizerCreationException("The Access Policy Provider must be set.");
-        }
-
-        accessPolicyProvider = accessPolicyProviderLookup.getAccessPolicyProvider(accessPolicyProviderKey.getValue());
-
-        // ensure the desired access policy provider was found
-        if (accessPolicyProvider == null) {
-            throw new AuthorizerCreationException(String.format("Unable to locate configured Access Policy Provider: %s", accessPolicyProviderKey));
-        }
-
-        userGroupProvider = accessPolicyProvider.getUserGroupProvider();
-
-        // ensure the desired access policy provider has a user group provider
-        if (userGroupProvider == null) {
-            throw new AuthorizerCreationException(String.format("Configured Access Policy Provider %s does not contain a User Group Provider", accessPolicyProviderKey));
-        }
-    }
-
-    @Override
-    public AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException {
-        final String resourceIdentifier = request.getResource().getIdentifier();
-        final AccessPolicy policy = accessPolicyProvider.getAccessPolicy(resourceIdentifier, request.getAction());
-        if (policy == null) {
-            return AuthorizationResult.resourceNotFound();
-        }
-
-        final UserAndGroups userAndGroups = userGroupProvider.getUserAndGroups(request.getIdentity());
-
-        final User user = userAndGroups.getUser();
-        if (user == null) {
-            return AuthorizationResult.denied(String.format("Unknown user with identity '%s'.", request.getIdentity()));
-        }
-
-        final Set<Group> userGroups = userAndGroups.getGroups();
-        if (policy.getUsers().contains(user.getIdentifier()) || containsGroup(userGroups, policy)) {
-            return AuthorizationResult.approved();
-        }
-
-        return AuthorizationResult.denied(request.getExplanationSupplier().get());
-    }
-
-    /**
-     * Determines if the policy contains one of the user's groups.
-     *
-     * @param userGroups the set of the user's groups
-     * @param policy the policy
-     * @return true if one of the Groups in userGroups is contained in the policy
-     */
-    private boolean containsGroup(final Set<Group> userGroups, final AccessPolicy policy) {
-        if (userGroups == null || userGroups.isEmpty() || policy.getGroups().isEmpty()) {
-            return false;
-        }
-
-        for (Group userGroup : userGroups) {
-            if (policy.getGroups().contains(userGroup.getIdentifier())) {
-                return true;
-            }
-        }
-
-        return false;
-    }
-
-    @Override
-    public String getFingerprint() throws AuthorizationAccessException {
-        XMLStreamWriter writer = null;
-        final StringWriter out = new StringWriter();
-        try {
-            writer = XML_OUTPUT_FACTORY.createXMLStreamWriter(out);
-            writer.writeStartDocument();
-            writer.writeStartElement("managedAuthorizations");
-
-            writer.writeStartElement(ACCESS_POLICY_PROVIDER_ELEMENT);
-            if (accessPolicyProvider instanceof ConfigurableAccessPolicyProvider) {
-                writer.writeCharacters(((ConfigurableAccessPolicyProvider) accessPolicyProvider).getFingerprint());
-            }
-            writer.writeEndElement();
-
-            writer.writeStartElement(USER_GROUP_PROVIDER_ELEMENT);
-            if (userGroupProvider instanceof ConfigurableUserGroupProvider) {
-                writer.writeCharacters(((ConfigurableUserGroupProvider) userGroupProvider).getFingerprint());
-            }
-            writer.writeEndElement();
-
-            writer.writeEndElement();
-            writer.writeEndDocument();
-            writer.flush();
-        } catch (XMLStreamException e) {
-            throw new AuthorizationAccessException("Unable to generate fingerprint", e);
-        } finally {
-            if (writer != null) {
-                try {
-                    writer.close();
-                } catch (XMLStreamException e) {
-                    // nothing to do here
-                }
-            }
-        }
-
-        return out.toString();
-    }
-
-    @Override
-    public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
-        if (StringUtils.isBlank(fingerprint)) {
-            return;
-        }
-
-        final FingerprintHolder fingerprintHolder = parseFingerprint(fingerprint);
-
-        if (StringUtils.isNotBlank(fingerprintHolder.getPolicyFingerprint()) && accessPolicyProvider instanceof ConfigurableAccessPolicyProvider) {
-            ((ConfigurableAccessPolicyProvider) accessPolicyProvider).inheritFingerprint(fingerprintHolder.getPolicyFingerprint());
-        }
-
-        if (StringUtils.isNotBlank(fingerprintHolder.getUserGroupFingerprint()) && userGroupProvider instanceof ConfigurableUserGroupProvider) {
-            ((ConfigurableUserGroupProvider) userGroupProvider).inheritFingerprint(fingerprintHolder.getUserGroupFingerprint());
-        }
-    }
-
-    @Override
-    public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
-        final FingerprintHolder fingerprintHolder = parseFingerprint(proposedFingerprint);
-
-        if (StringUtils.isNotBlank(fingerprintHolder.getPolicyFingerprint())) {
-            if (accessPolicyProvider instanceof ConfigurableAccessPolicyProvider) {
-                ((ConfigurableAccessPolicyProvider) accessPolicyProvider).checkInheritability(fingerprintHolder.getPolicyFingerprint());
-            } else {
-                throw new UninheritableAuthorizationsException("Policy fingerprint is not blank and the configured AccessPolicyProvider does not support fingerprinting.");
-            }
-        }
-
-        if (StringUtils.isNotBlank(fingerprintHolder.getUserGroupFingerprint())) {
-            if (userGroupProvider instanceof ConfigurableUserGroupProvider) {
-                ((ConfigurableUserGroupProvider) userGroupProvider).checkInheritability(fingerprintHolder.getUserGroupFingerprint());
-            } else {
-                throw new UninheritableAuthorizationsException("User/Group fingerprint is not blank and the configured UserGroupProvider does not support fingerprinting.");
-            }
-        }
-    }
-
-    private final FingerprintHolder parseFingerprint(final String fingerprint) throws AuthorizationAccessException {
-        final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
-
-        try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
-            final DocumentBuilder docBuilder = DOCUMENT_BUILDER_FACTORY.newDocumentBuilder();
-            final Document document = docBuilder.parse(in);
-            final Element rootElement = document.getDocumentElement();
-
-            final NodeList accessPolicyProviderList = rootElement.getElementsByTagName(ACCESS_POLICY_PROVIDER_ELEMENT);
-            if (accessPolicyProviderList.getLength() != 1) {
-                throw new AuthorizationAccessException(String.format("Only one %s element is allowed: %s", ACCESS_POLICY_PROVIDER_ELEMENT, fingerprint));
-            }
-
-            final NodeList userGroupProviderList = rootElement.getElementsByTagName(USER_GROUP_PROVIDER_ELEMENT);
-            if (userGroupProviderList.getLength() != 1) {
-                throw new AuthorizationAccessException(String.format("Only one %s element is allowed: %s", USER_GROUP_PROVIDER_ELEMENT, fingerprint));
-            }
-
-            final Node accessPolicyProvider = accessPolicyProviderList.item(0);
-            final Node userGroupProvider = userGroupProviderList.item(0);
-            return new FingerprintHolder(accessPolicyProvider.getTextContent(), userGroupProvider.getTextContent());
-        } catch (SAXException | ParserConfigurationException | IOException e) {
-            throw new AuthorizationAccessException("Unable to parse fingerprint", e);
-        }
-    }
-
-    @Override
-    public AccessPolicyProvider getAccessPolicyProvider() {
-        return accessPolicyProvider;
-    }
-
-    @Override
-    public void preDestruction() throws AuthorizerDestructionException {
-
-    }
-
-    private static class FingerprintHolder {
-        private final String policyFingerprint;
-        private final String userGroupFingerprint;
-
-        public FingerprintHolder(String policyFingerprint, String userGroupFingerprint) {
-            this.policyFingerprint = policyFingerprint;
-            this.userGroupFingerprint = userGroupFingerprint;
-        }
-
-        public String getPolicyFingerprint() {
-            return policyFingerprint;
-        }
-
-        public String getUserGroupFingerprint() {
-            return userGroupFingerprint;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/AccessPolicyAuthorizable.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/AccessPolicyAuthorizable.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/AccessPolicyAuthorizable.java
deleted file mode 100644
index 5d9997f..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/AccessPolicyAuthorizable.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.resource;
-
-import org.apache.nifi.registry.authorization.exception.AccessDeniedException;
-import org.apache.nifi.registry.authorization.AuthorizationResult;
-import org.apache.nifi.registry.authorization.AuthorizationResult.Result;
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.RequestAction;
-import org.apache.nifi.registry.authorization.Resource;
-import org.apache.nifi.registry.authorization.user.NiFiUser;
-
-import java.util.Map;
-
-/**
- * Authorizable for policies of an Authorizable.
- */
-public class AccessPolicyAuthorizable implements Authorizable, EnforcePolicyPermissionsThroughBaseResource {
-
-    private static final Authorizable POLICIES_AUTHORIZABLE = new Authorizable() {
-        @Override
-        public Authorizable getParentAuthorizable() {
-            return null;
-        }
-
-        @Override
-        public Resource getResource() {
-            return ResourceFactory.getPoliciesResource();
-        }
-    };
-
-    final Authorizable authorizable;
-
-    public AccessPolicyAuthorizable(Authorizable authorizable) {
-        this.authorizable = authorizable;
-    }
-
-    @Override
-    public Authorizable getBaseAuthorizable() {
-        return authorizable;
-    }
-
-    @Override
-    public Authorizable getParentAuthorizable() {
-        final Authorizable effectiveAuthorizable = getEffectiveAuthorizable();
-        if (effectiveAuthorizable.getParentAuthorizable() == null) {
-            return POLICIES_AUTHORIZABLE;
-        } else {
-            return new AccessPolicyAuthorizable(effectiveAuthorizable.getParentAuthorizable());
-        }
-    }
-
-    @Override
-    public Resource getResource() {
-        return ResourceFactory.getPolicyResource(getEffectiveAuthorizable().getResource());
-    }
-
-    private Authorizable getEffectiveAuthorizable() {
-        // possibly consider the base resource if the authorizable uses it to enforce policy permissions
-        if (authorizable instanceof EnforcePolicyPermissionsThroughBaseResource) {
-            final Authorizable baseAuthorizable = ((EnforcePolicyPermissionsThroughBaseResource) authorizable).getBaseAuthorizable();
-
-            // if the base authorizable is for a policy, we don't want to use the base otherwise it would keep unwinding and would eventually
-            // evaluate to the policy of the component and not the policy of the policies for the component
-            if (baseAuthorizable instanceof AccessPolicyAuthorizable) {
-                return authorizable;
-            } else {
-                return baseAuthorizable;
-            }
-        } else {
-            return authorizable;
-        }
-    }
-
-    @Override
-    public AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) {
-        if (user == null) {
-            throw new AccessDeniedException("Unknown user.");
-        }
-
-        final AuthorizationResult resourceResult = Authorizable.super.checkAuthorization(authorizer, action, user, resourceContext);
-
-        // if we're denied from the resource try inheriting
-        if (Result.Denied.equals(resourceResult.getResult())) {
-            return getParentAuthorizable().checkAuthorization(authorizer, action, user, resourceContext);
-        } else {
-            return resourceResult;
-        }
-    }
-
-    @Override
-    public void authorize(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) throws AccessDeniedException {
-        if (user == null) {
-            throw new AccessDeniedException("Unknown user.");
-        }
-
-        try {
-            Authorizable.super.authorize(authorizer, action, user, resourceContext);
-        } catch (final AccessDeniedException resourceDenied) {
-            // if we're denied from the resource try inheriting
-            try {
-                getParentAuthorizable().authorize(authorizer, action, user, resourceContext);
-            } catch (final AccessDeniedException policiesDenied) {
-                throw resourceDenied;
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/Authorizable.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/Authorizable.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/Authorizable.java
deleted file mode 100644
index 77f0fec..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/Authorizable.java
+++ /dev/null
@@ -1,300 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.resource;
-
-import org.apache.nifi.registry.authorization.AuthorizationResult.Result;
-import org.apache.nifi.registry.authorization.exception.AccessDeniedException;
-import org.apache.nifi.registry.authorization.user.NiFiUser;
-import org.apache.nifi.registry.authorization.AuthorizationAuditor;
-import org.apache.nifi.registry.authorization.AuthorizationRequest;
-import org.apache.nifi.registry.authorization.AuthorizationResult;
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.RequestAction;
-import org.apache.nifi.registry.authorization.Resource;
-import org.apache.nifi.registry.authorization.UserContextKeys;
-
-import java.util.HashMap;
-import java.util.Map;
-
-public interface Authorizable {
-
-    /**
-     * The parent for this Authorizable. May be null.
-     *
-     * @return the parent authorizable or null
-     */
-    Authorizable getParentAuthorizable();
-
-    /**
-     * The Resource for this Authorizable.
-     *
-     * @return the resource
-     */
-    Resource getResource();
-
-    /**
-     * The originally requested resource for this Authorizable. Because policies are inherited, if a resource
-     * does not have a policy, this Authorizable may represent a parent resource and this method will return
-     * the originally requested resource.
-     *
-     * @return the originally requested resource
-     */
-    default Resource getRequestedResource() {
-        return getResource();
-    }
-
-    /**
-     * Returns whether the current user is authorized for the specified action on the specified resource. This
-     * method does not imply the user is directly attempting to access the specified resource. If the user is
-     * attempting a direct access use Authorizable.authorize().
-     *
-     * @param authorizer authorizer
-     * @param action action
-     * @return is authorized
-     */
-    default boolean isAuthorized(Authorizer authorizer, RequestAction action, NiFiUser user) {
-        return Result.Approved.equals(checkAuthorization(authorizer, action, user).getResult());
-    }
-
-    /**
-     * Returns the result of an authorization request for the specified user for the specified action on the specified
-     * resource. This method does not imply the user is directly attempting to access the specified resource. If the user is
-     * attempting a direct access use Authorizable.authorize().
-     *
-     * @param authorizer authorizer
-     * @param action action
-     * @param user user
-     * @return is authorized
-     */
-    default AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) {
-        if (user == null) {
-            return AuthorizationResult.denied("Unknown user.");
-        }
-
-        final Map<String,String> userContext;
-        if (user.getClientAddress() != null && !user.getClientAddress().trim().isEmpty()) {
-            userContext = new HashMap<>();
-            userContext.put(UserContextKeys.CLIENT_ADDRESS.name(), user.getClientAddress());
-        } else {
-            userContext = null;
-        }
-
-        final Resource resource = getResource();
-        final Resource requestedResource = getRequestedResource();
-        final AuthorizationRequest request = new AuthorizationRequest.Builder()
-                .identity(user.getIdentity())
-                .groups(user.getGroups())
-                .anonymous(user.isAnonymous())
-                .accessAttempt(false)
-                .action(action)
-                .resource(resource)
-                .requestedResource(requestedResource)
-                .resourceContext(resourceContext)
-                .userContext(userContext)
-                .explanationSupplier(() -> {
-                    // build the safe explanation
-                    final StringBuilder safeDescription = new StringBuilder("Unable to ");
-
-                    if (RequestAction.READ.equals(action)) {
-                        safeDescription.append("view ");
-                    } else {
-                        safeDescription.append("modify "); // covers write or delete
-                    }
-                    safeDescription.append(resource.getSafeDescription()).append(".");
-
-                    return safeDescription.toString();
-                })
-                .build();
-
-        // perform the authorization
-        final AuthorizationResult result = authorizer.authorize(request);
-
-        // verify the results
-        if (Result.ResourceNotFound.equals(result.getResult())) {
-            final Authorizable parent = getParentAuthorizable();
-            if (parent == null) {
-                return AuthorizationResult.denied("No applicable policies could be found.");
-            } else {
-                // create a custom authorizable to override the safe description but still defer to the parent authorizable
-                final Authorizable parentProxy = new Authorizable() {
-                    @Override
-                    public Authorizable getParentAuthorizable() {
-                        return parent.getParentAuthorizable();
-                    }
-
-                    @Override
-                    public Resource getRequestedResource() {
-                        return requestedResource;
-                    }
-
-                    @Override
-                    public Resource getResource() {
-                        final Resource parentResource = parent.getResource();
-                        return new Resource() {
-                            @Override
-                            public String getIdentifier() {
-                                return parentResource.getIdentifier();
-                            }
-
-                            @Override
-                            public String getName() {
-                                return parentResource.getName();
-                            }
-
-                            @Override
-                            public String getSafeDescription() {
-                                return resource.getSafeDescription();
-                            }
-                        };
-                    }
-                };
-                return parentProxy.checkAuthorization(authorizer, action, user, resourceContext);
-            }
-        } else {
-            return result;
-        }
-    }
-
-    /**
-     * Returns the result of an authorization request for the specified user for the specified action on the specified
-     * resource. This method does not imply the user is directly attempting to access the specified resource. If the user is
-     * attempting a direct access use Authorizable.authorize().
-     *
-     * @param authorizer authorizer
-     * @param action action
-     * @param user user
-     * @return is authorized
-     */
-    default AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action, NiFiUser user) {
-        return checkAuthorization(authorizer, action, user, null);
-    }
-
-    /**
-     * Authorizes the current user for the specified action on the specified resource. This method does imply the user is
-     * directly accessing the specified resource.
-     *
-     * @param authorizer authorizer
-     * @param action action
-     * @param user user
-     * @param resourceContext resource context
-     */
-    default void authorize(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) throws AccessDeniedException {
-        if (user == null) {
-            throw new AccessDeniedException("Unknown user.");
-        }
-
-        final Map<String,String> userContext;
-        if (user.getClientAddress() != null && !user.getClientAddress().trim().isEmpty()) {
-            userContext = new HashMap<>();
-            userContext.put(UserContextKeys.CLIENT_ADDRESS.name(), user.getClientAddress());
-        } else {
-            userContext = null;
-        }
-
-        final Resource resource = getResource();
-        final Resource requestedResource = getRequestedResource();
-        final AuthorizationRequest request = new AuthorizationRequest.Builder()
-                .identity(user.getIdentity())
-                .groups(user.getGroups())
-                .anonymous(user.isAnonymous())
-                .accessAttempt(true)
-                .action(action)
-                .resource(resource)
-                .requestedResource(requestedResource)
-                .resourceContext(resourceContext)
-                .userContext(userContext)
-                .explanationSupplier(() -> {
-                    // build the safe explanation
-                    final StringBuilder safeDescription = new StringBuilder("Unable to ");
-
-                    if (RequestAction.READ.equals(action)) {
-                        safeDescription.append("view ");
-                    } else {
-                        safeDescription.append("modify ");
-                    }
-                    safeDescription.append(resource.getSafeDescription()).append(".");
-
-                    return safeDescription.toString();
-                })
-                .build();
-
-        final AuthorizationResult result = authorizer.authorize(request);
-        if (Result.ResourceNotFound.equals(result.getResult())) {
-            final Authorizable parent = getParentAuthorizable();
-            if (parent == null) {
-                final AuthorizationResult failure = AuthorizationResult.denied("No applicable policies could be found.");
-
-                // audit authorization request
-                if (authorizer instanceof AuthorizationAuditor) {
-                    ((AuthorizationAuditor) authorizer).auditAccessAttempt(request, failure);
-                }
-
-                // denied
-                throw new AccessDeniedException(failure.getExplanation());
-            } else {
-                // create a custom authorizable to override the safe description but still defer to the parent authorizable
-                final Authorizable parentProxy = new Authorizable() {
-                    @Override
-                    public Authorizable getParentAuthorizable() {
-                        return parent.getParentAuthorizable();
-                    }
-
-                    @Override
-                    public Resource getRequestedResource() {
-                        return requestedResource;
-                    }
-
-                    @Override
-                    public Resource getResource() {
-                        final Resource parentResource = parent.getResource();
-                        return new Resource() {
-                            @Override
-                            public String getIdentifier() {
-                                return parentResource.getIdentifier();
-                            }
-
-                            @Override
-                            public String getName() {
-                                return parentResource.getName();
-                            }
-
-                            @Override
-                            public String getSafeDescription() {
-                                return resource.getSafeDescription();
-                            }
-                        };
-                    }
-                };
-                parentProxy.authorize(authorizer, action, user, resourceContext);
-            }
-        } else if (Result.Denied.equals(result.getResult())) {
-            throw new AccessDeniedException(result.getExplanation());
-        }
-    }
-
-    /**
-     * Authorizes the current user for the specified action on the specified resource. This method does imply the user is
-     * directly accessing the specified resource.
-     *
-     * @param authorizer authorizer
-     * @param action action
-     * @param user user
-     */
-    default void authorize(Authorizer authorizer, RequestAction action, NiFiUser user) throws AccessDeniedException {
-        authorize(authorizer, action, user, null);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/EnforcePolicyPermissionsThroughBaseResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/EnforcePolicyPermissionsThroughBaseResource.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/EnforcePolicyPermissionsThroughBaseResource.java
deleted file mode 100644
index 2d6b1a8..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/EnforcePolicyPermissionsThroughBaseResource.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.resource;
-
-/**
- * Defers permissions on policies to the policies of the base authorizable. Required because we don't
- * want to change the enforcement of the policies on the authorizable. For example...
- *
- * if a user has permissions to /policies/input-ports/1234 then they have permissions to the following
- *
- * - the policy for /buckets/1234                    -> /policies/buckets/1234
- * - the policy for /policies/buckets/1234           -> /policies/policies/buckets/1234
- */
-public interface EnforcePolicyPermissionsThroughBaseResource {
-
-    /**
-     * Returns the base authorizable. Cannot be null.
-     *
-     * @return base authorizable
-     */
-    Authorizable getBaseAuthorizable();
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/ResourceFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/ResourceFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/ResourceFactory.java
deleted file mode 100644
index 1d0b023..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/ResourceFactory.java
+++ /dev/null
@@ -1,261 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.resource;
-
-import org.apache.nifi.registry.authorization.Resource;
-
-import java.util.Objects;
-
-public final class ResourceFactory {
-
-    private final static Resource BUCKETS_RESOURCE = new Resource() {
-        @Override
-        public String getIdentifier() {
-            return ResourceType.Bucket.getValue();
-        }
-
-        @Override
-        public String getName() {
-            return "Buckets";
-        }
-
-        @Override
-        public String getSafeDescription() {
-            return "buckets";
-        }
-    };
-
-
-    private final static Resource POLICY_RESOURCE = new Resource() {
-        @Override
-        public String getIdentifier() {
-            return ResourceType.Policy.getValue();
-        }
-
-        @Override
-        public String getName() {
-            return "Policies for ";
-        }
-
-        @Override
-        public String getSafeDescription() {
-            return "the policies for ";
-        }
-    };
-
-    private final static Resource PROXY_RESOURCE = new Resource() {
-        @Override
-        public String getIdentifier() {
-            return ResourceType.Proxy.getValue();
-        }
-
-        @Override
-        public String getName() {
-            return "Proxy User Requests";
-        }
-
-        @Override
-        public String getSafeDescription() {
-            return "proxy requests on behalf of users";
-        }
-    };
-
-    private final static Resource RESOURCE_RESOURCE = new Resource() {
-        @Override
-        public String getIdentifier() {
-            return ResourceType.Resource.getValue();
-        }
-
-        @Override
-        public String getName() {
-            return "NiFi Resources";
-        }
-
-        @Override
-        public String getSafeDescription() {
-            return "resources";
-        }
-    };
-
-    private final static Resource TENANT_RESOURCE = new Resource() {
-        @Override
-        public String getIdentifier() {
-            return ResourceType.Tenant.getValue();
-        }
-
-        @Override
-        public String getName() {
-            return "Tenant";
-        }
-
-        @Override
-        public String getSafeDescription() {
-            return "users/user groups";
-        }
-    };
-
-    private final static Resource POLICIES_RESOURCE = new Resource() {
-
-        @Override
-        public String getIdentifier() {
-            return "/policies";
-        }
-
-        @Override
-        public String getName() {
-            return "Access Policies";
-        }
-
-        @Override
-        public String getSafeDescription() {
-            return "policies";
-        }
-    };
-
-
-    /**
-     * Gets the Resource for proxying a user request.
-     *
-     * @return  The resource for proxying a user request
-     */
-    public static Resource getProxyResource() {
-        return PROXY_RESOURCE;
-    }
-
-    /**
-     * Gets the Resource for detailing all available NiFi Resources.
-     *
-     * @return  The Resource resource
-     */
-    public static Resource getResourceResource() {
-        return RESOURCE_RESOURCE;
-    }
-
-    /**
-     * Gets the Resource for accessing Tenants which includes creating, modifying, and deleting Users and UserGroups.
-     *
-     * @return The Resource for accessing Tenants
-     */
-    public static Resource getTenantResource() {
-        return TENANT_RESOURCE;
-    }
-
-    /**
-     * Gets the {@link Resource} for accessing access policies.
-     * @return The policies resource
-     */
-    public static Resource getPoliciesResource() {
-        return POLICIES_RESOURCE;
-    }
-
-    /**
-     * Gets the {@link Resource} for accessing buckets.
-     * @return The buckets resource
-     */
-    public static Resource getBucketsResource() {
-        return BUCKETS_RESOURCE;
-    }
-
-    /**
-     * Gets the {@link Resource} for accessing buckets.
-     * @return The buckets resource
-     */
-    public static Resource getBucketResource(String bucketIdentifier, String bucketName) {
-        return getChildResource(ResourceType.Bucket, bucketIdentifier, bucketName);
-    }
-
-    /**
-     * Gets a Resource for accessing a resources's policies.
-     *
-     * @param resource      The resource being accessed
-     * @return              The resource
-     */
-    public static Resource getPolicyResource(final Resource resource) {
-        Objects.requireNonNull(resource, "The resource type must be specified.");
-
-        return new Resource() {
-            @Override
-            public String getIdentifier() {
-                return String.format("%s%s", POLICY_RESOURCE.getIdentifier(), resource.getIdentifier());
-            }
-
-            @Override
-            public String getName() {
-                return POLICY_RESOURCE.getName() + resource.getName();
-            }
-
-            @Override
-            public String getSafeDescription() {
-                return POLICY_RESOURCE.getSafeDescription() + resource.getSafeDescription();
-            }
-        };
-    }
-
-    /**
-     * Get a Resource object for any object that has a base type and an identifier, ie:
-     * /buckets/{uuid}
-     *
-     * @param parentResourceType - Required, the base resource type
-     * @param childIdentifier - Required, the identity of this sub resource
-     * @param name - Optional, the name of the subresource
-     * @return A resource for this object
-     */
-    public static Resource getChildResource(final ResourceType parentResourceType, final String childIdentifier, final String name) {
-        Objects.requireNonNull(parentResourceType, "The base resource type must be specified.");
-        Objects.requireNonNull(childIdentifier, "The child identifier identifier must be specified.");
-
-        return new Resource() {
-            @Override
-            public String getIdentifier() {
-                return String.format("%s/%s", parentResourceType.getValue(), childIdentifier);
-            }
-
-            @Override
-            public String getName() {
-                return name;
-            }
-
-            @Override
-            public String getSafeDescription() {
-                final StringBuilder safeDescription = new StringBuilder();
-                switch (parentResourceType) {
-                    case Bucket:
-                        safeDescription.append("Bucket");
-                        break;
-                    case Policy:
-                        safeDescription.append("Policy");
-                        break;
-                    case Tenant:
-                        safeDescription.append("Tenant");
-                        break;
-                    default:
-                        safeDescription.append("Unknown resource type");
-                        break;
-                }
-                safeDescription.append(" with ID ");
-                safeDescription.append(childIdentifier);
-                return safeDescription.toString();
-            }
-        };
-
-    }
-
-    /**
-     * Prevent outside instantiation.
-     */
-    private ResourceFactory() {}
-}


[02/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/NiFiAuthenticationToken.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/NiFiAuthenticationToken.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/NiFiAuthenticationToken.java
new file mode 100644
index 0000000..19e56c5
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/token/NiFiAuthenticationToken.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.web.security.authentication.token;
+
+import org.springframework.security.authentication.AbstractAuthenticationToken;
+import org.springframework.security.core.userdetails.UserDetails;
+
+/**
+ * An authentication token that represents an Authenticated and Authorized user of the NiFi Apis. The authorities are based off the specified UserDetails.
+ */
+public class NiFiAuthenticationToken extends AbstractAuthenticationToken {
+
+    final UserDetails nifiUserDetails;
+
+    public NiFiAuthenticationToken(final UserDetails nifiUserDetails) {
+        super(nifiUserDetails.getAuthorities());
+        super.setAuthenticated(true);
+        setDetails(nifiUserDetails);
+        this.nifiUserDetails = nifiUserDetails;
+    }
+
+    @Override
+    public Object getCredentials() {
+        return nifiUserDetails.getPassword();
+    }
+
+    @Override
+    public Object getPrincipal() {
+        return nifiUserDetails;
+    }
+
+    @Override
+    public final void setAuthenticated(boolean authenticated) {
+        throw new IllegalArgumentException("Cannot change the authenticated state.");
+    }
+
+    @Override
+    public String toString() {
+        return nifiUserDetails.getUsername();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/SubjectDnX509PrincipalExtractor.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/SubjectDnX509PrincipalExtractor.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/SubjectDnX509PrincipalExtractor.java
new file mode 100644
index 0000000..a9deae1
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/SubjectDnX509PrincipalExtractor.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.web.security.authentication.x509;
+
+import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
+import org.springframework.stereotype.Component;
+
+import java.security.cert.X509Certificate;
+
+/**
+ * Principal extractor for extracting a DN.
+ */
+@Component
+public class SubjectDnX509PrincipalExtractor implements X509PrincipalExtractor {
+
+    @Override
+    public Object extractPrincipal(X509Certificate cert) {
+        return cert.getSubjectDN().getName().trim();
+    }
+
+}

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationRequestToken.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationRequestToken.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationRequestToken.java
new file mode 100644
index 0000000..d5aca23
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509AuthenticationRequestToken.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.web.security.authentication.x509;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.web.security.authentication.NiFiAuthenticationRequestToken;
+import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
+
+import java.security.cert.X509Certificate;
+
+/**
+ * This is an authentication request with a given JWT token.
+ */
+public class X509AuthenticationRequestToken extends NiFiAuthenticationRequestToken {
+
+    private final String proxiedEntitiesChain;
+    private final X509PrincipalExtractor principalExtractor;
+    private final X509Certificate[] certificates;
+
+    /**
+     * Creates a representation of the jwt authentication request for a user.
+     *
+     * @param proxiedEntitiesChain   The http servlet request
+     * @param certificates  The certificate chain
+     */
+    public X509AuthenticationRequestToken(final String proxiedEntitiesChain, final X509PrincipalExtractor principalExtractor, final X509Certificate[] certificates, final String clientAddress) {
+        super(clientAddress);
+        setAuthenticated(false);
+        this.proxiedEntitiesChain = proxiedEntitiesChain;
+        this.principalExtractor = principalExtractor;
+        this.certificates = certificates;
+    }
+
+    @Override
+    public Object getCredentials() {
+        return null;
+    }
+
+    @Override
+    public Object getPrincipal() {
+        if (StringUtils.isBlank(proxiedEntitiesChain)) {
+            return principalExtractor.extractPrincipal(certificates[0]);
+        } else {
+            return String.format("%s<%s>", proxiedEntitiesChain, principalExtractor.extractPrincipal(certificates[0]));
+        }
+    }
+
+    public String getProxiedEntitiesChain() {
+        return proxiedEntitiesChain;
+    }
+
+    public X509Certificate[] getCertificates() {
+        return certificates;
+    }
+
+    @Override
+    public String toString() {
+        return getName();
+    }
+
+}

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/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
new file mode 100644
index 0000000..692b318
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/x509/X509IdentityProvider.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.web.security.authentication.x509;
+
+import org.apache.nifi.registry.web.response.AuthenticationResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
+import org.springframework.stereotype.Component;
+
+import java.security.cert.CertificateExpiredException;
+import java.security.cert.CertificateNotYetValidException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Identity provider for extract the authenticating a ServletRequest with a X509Certificate.
+ */
+@Component
+public class X509IdentityProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(X509IdentityProvider.class);
+
+    private final String issuer = getClass().getSimpleName();
+
+    private X509CertificateValidator certificateValidator;
+    private X509PrincipalExtractor principalExtractor;
+
+    @Autowired
+    public X509IdentityProvider(X509CertificateValidator certificateValidator, X509PrincipalExtractor principalExtractor) {
+        this.certificateValidator = certificateValidator;
+        this.principalExtractor = principalExtractor;
+    }
+
+    /**
+     * Authenticates the specified request by checking certificate validity.
+     *
+     * @param certificates the client certificates
+     * @return an authentication response
+     * @throws IllegalArgumentException the request did not contain a valid certificate (or no certificate)
+     */
+    public AuthenticationResponse authenticate(final X509Certificate[] certificates) throws IllegalArgumentException {
+        // ensure the cert was found
+        if (certificates == null || certificates.length == 0) {
+            throw new IllegalArgumentException("The specified request does not contain a client certificate.");
+        }
+
+        // extract the principal
+        final Object certificatePrincipal = principalExtractor.extractPrincipal(certificates[0]);
+        final String principal = certificatePrincipal.toString();
+
+        try {
+            certificateValidator.validateClientCertificate(certificates);
+        } catch (CertificateExpiredException cee) {
+            final String message = String.format("Client certificate for (%s) is expired.", principal);
+            logger.info(message, cee);
+            if (logger.isDebugEnabled()) {
+                logger.debug("", cee);
+            }
+            throw new IllegalArgumentException(message, cee);
+        } catch (CertificateNotYetValidException cnyve) {
+            final String message = String.format("Client certificate for (%s) is not yet valid.", principal);
+            logger.info(message, cnyve);
+            if (logger.isDebugEnabled()) {
+                logger.debug("", cnyve);
+            }
+            throw new IllegalArgumentException(message, cnyve);
+        } catch (final Exception e) {
+            logger.info(e.getMessage());
+            if (logger.isDebugEnabled()) {
+                logger.debug("", e);
+            }
+            throw new IllegalArgumentException(e.getMessage(), e);
+        }
+
+        // build the authentication response
+        return new AuthenticationResponse(principal, principal, TimeUnit.MILLISECONDS.convert(12, TimeUnit.HOURS), issuer);
+    }
+
+    /* setters */
+    public void setCertificateValidator(X509CertificateValidator certificateValidator) {
+        this.certificateValidator = certificateValidator;
+    }
+
+    public void setPrincipalExtractor(X509PrincipalExtractor principalExtractor) {
+        this.principalExtractor = principalExtractor;
+    }
+
+}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/token/NiFiAuthenticationToken.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/token/NiFiAuthenticationToken.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/token/NiFiAuthenticationToken.java
deleted file mode 100644
index 5ea9e5b..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/token/NiFiAuthenticationToken.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.token;
-
-import org.springframework.security.authentication.AbstractAuthenticationToken;
-import org.springframework.security.core.userdetails.UserDetails;
-
-/**
- * An authentication token that represents an Authenticated and Authorized user of the NiFi Apis. The authorities are based off the specified UserDetails.
- */
-public class NiFiAuthenticationToken extends AbstractAuthenticationToken {
-
-    final UserDetails nifiUserDetails;
-
-    public NiFiAuthenticationToken(final UserDetails nifiUserDetails) {
-        super(nifiUserDetails.getAuthorities());
-        super.setAuthenticated(true);
-        setDetails(nifiUserDetails);
-        this.nifiUserDetails = nifiUserDetails;
-    }
-
-    @Override
-    public Object getCredentials() {
-        return nifiUserDetails.getPassword();
-    }
-
-    @Override
-    public Object getPrincipal() {
-        return nifiUserDetails;
-    }
-
-    @Override
-    public final void setAuthenticated(boolean authenticated) {
-        throw new IllegalArgumentException("Cannot change the authenticated state.");
-    }
-
-    @Override
-    public String toString() {
-        return nifiUserDetails.getUsername();
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/token/OtpAuthenticationToken.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/token/OtpAuthenticationToken.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/token/OtpAuthenticationToken.java
deleted file mode 100644
index f49e97d..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/token/OtpAuthenticationToken.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.token;
-
-import org.springframework.security.authentication.AbstractAuthenticationToken;
-
-/**
- * This is an Authentication Token for logging in. Once a user is authenticated, they can be issued an ID token.
- */
-public class OtpAuthenticationToken extends AbstractAuthenticationToken {
-
-    private final String identity;
-
-    /**
-     * Creates a representation of the otp authentication token for a user.
-     *
-     * @param identity   The unique identifier for this user
-     */
-    public OtpAuthenticationToken(final String identity) {
-        super(null);
-        setAuthenticated(true);
-        this.identity = identity;
-    }
-
-    @Override
-    public Object getCredentials() {
-        return null;
-    }
-
-    @Override
-    public Object getPrincipal() {
-        return identity;
-    }
-
-    @Override
-    public String toString() {
-        return new StringBuilder("OtpAuthenticationToken for ")
-                .append(getName())
-                .toString();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/SubjectDnX509PrincipalExtractor.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/SubjectDnX509PrincipalExtractor.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/SubjectDnX509PrincipalExtractor.java
deleted file mode 100644
index f8ab1fe..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/SubjectDnX509PrincipalExtractor.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.x509;
-
-import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
-
-import java.security.cert.X509Certificate;
-
-/**
- * Principal extractor for extracting a DN.
- */
-public class SubjectDnX509PrincipalExtractor implements X509PrincipalExtractor {
-
-    @Override
-    public Object extractPrincipal(X509Certificate cert) {
-        return cert.getSubjectDN().getName().trim();
-    }
-
-}

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

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/X509CertificateExtractor.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/X509CertificateExtractor.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/X509CertificateExtractor.java
deleted file mode 100644
index ad62b37..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/X509CertificateExtractor.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.x509;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import javax.servlet.http.HttpServletRequest;
-import java.security.cert.X509Certificate;
-
-/**
- * Extracts client certificates from Http requests.
- */
-public class X509CertificateExtractor {
-
-    private final Logger logger = LoggerFactory.getLogger(getClass());
-
-    /**
-     * Extract the client certificate from the specified HttpServletRequest or
-     * null if none is specified.
-     *
-     * @param request http request
-     * @return cert
-     */
-    public X509Certificate[] extractClientCertificate(HttpServletRequest request) {
-        X509Certificate[] certs = (X509Certificate[]) request.getAttribute("javax.servlet.request.X509Certificate");
-
-        if (certs != null && certs.length > 0) {
-            return certs;
-        }
-
-        if (logger.isDebugEnabled()) {
-            logger.debug("No client certificate found in request.");
-        }
-
-        return null;
-    }
-
-}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/X509IdentityProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/X509IdentityProvider.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/X509IdentityProvider.java
deleted file mode 100644
index 38aa0b4..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/x509/X509IdentityProvider.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security.x509;
-
-import org.apache.nifi.registry.web.response.AuthenticationResponse;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
-
-import java.security.cert.CertificateExpiredException;
-import java.security.cert.CertificateNotYetValidException;
-import java.security.cert.X509Certificate;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Identity provider for extract the authenticating a ServletRequest with a X509Certificate.
- */
-public class X509IdentityProvider {
-
-    private static final Logger logger = LoggerFactory.getLogger(X509IdentityProvider.class);
-
-    private final String issuer = getClass().getSimpleName();
-
-    private X509CertificateValidator certificateValidator;
-    private X509PrincipalExtractor principalExtractor;
-
-    /**
-     * Authenticates the specified request by checking certificate validity.
-     *
-     * @param certificates the client certificates
-     * @return an authentication response
-     * @throws IllegalArgumentException the request did not contain a valid certificate (or no certificate)
-     */
-    public AuthenticationResponse authenticate(final X509Certificate[] certificates) throws IllegalArgumentException {
-        // ensure the cert was found
-        if (certificates == null || certificates.length == 0) {
-            throw new IllegalArgumentException("The specified request does not contain a client certificate.");
-        }
-
-        // extract the principal
-        final Object certificatePrincipal = principalExtractor.extractPrincipal(certificates[0]);
-        final String principal = certificatePrincipal.toString();
-
-        try {
-            certificateValidator.validateClientCertificate(certificates);
-        } catch (CertificateExpiredException cee) {
-            final String message = String.format("Client certificate for (%s) is expired.", principal);
-            logger.info(message, cee);
-            if (logger.isDebugEnabled()) {
-                logger.debug("", cee);
-            }
-            throw new IllegalArgumentException(message, cee);
-        } catch (CertificateNotYetValidException cnyve) {
-            final String message = String.format("Client certificate for (%s) is not yet valid.", principal);
-            logger.info(message, cnyve);
-            if (logger.isDebugEnabled()) {
-                logger.debug("", cnyve);
-            }
-            throw new IllegalArgumentException(message, cnyve);
-        } catch (final Exception e) {
-            logger.info(e.getMessage());
-            if (logger.isDebugEnabled()) {
-                logger.debug("", e);
-            }
-            throw new IllegalArgumentException(e.getMessage(), e);
-        }
-
-        // build the authentication response
-        return new AuthenticationResponse(principal, principal, TimeUnit.MILLISECONDS.convert(12, TimeUnit.HOURS), issuer);
-    }
-
-    /* setters */
-    public void setCertificateValidator(X509CertificateValidator certificateValidator) {
-        this.certificateValidator = certificateValidator;
-    }
-
-    public void setPrincipalExtractor(X509PrincipalExtractor principalExtractor) {
-        this.principalExtractor = principalExtractor;
-    }
-
-}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/NiFiRegistryApiTestApplication.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/NiFiRegistryApiTestApplication.java b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/NiFiRegistryApiTestApplication.java
deleted file mode 100644
index 01b0269..0000000
--- a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/NiFiRegistryApiTestApplication.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry;
-
-import org.apache.nifi.registry.db.DataSourceFactory;
-import org.springframework.boot.SpringApplication;
-import org.springframework.boot.autoconfigure.SpringBootApplication;
-import org.springframework.boot.builder.SpringApplicationBuilder;
-import org.springframework.boot.web.support.SpringBootServletInitializer;
-import org.springframework.context.annotation.ComponentScan;
-import org.springframework.context.annotation.FilterType;
-
-import java.util.Properties;
-
-@SpringBootApplication
-@ComponentScan(
-        excludeFilters = {
-                @ComponentScan.Filter(
-                        type = FilterType.ASSIGNABLE_TYPE,
-                        value = SpringBootServletInitializer.class), // Avoid loading NiFiRegistryApiApplication
-                @ComponentScan.Filter(
-                        type = FilterType.ASSIGNABLE_TYPE,
-                        value = DataSourceFactory.class), // Avoid loading DataSourceFactory
-                @ComponentScan.Filter(
-                        type = FilterType.REGEX,
-                        pattern = "org\\.apache\\.nifi\\.registry\\.NiFiRegistryPropertiesFactory"), // Avoid loading NiFiRegistryPropertiesFactory
-        })
-public class NiFiRegistryApiTestApplication extends SpringBootServletInitializer {
-
-    @Override
-    protected SpringApplicationBuilder configure(SpringApplicationBuilder application) {
-        final Properties fixedProps = new Properties();
-        fixedProps.setProperty("spring.jpa.hibernate.ddl-auto", "none");
-        fixedProps.setProperty("spring.jpa.hibernate.naming.physical-strategy", "org.hibernate.boot.model.naming.PhysicalNamingStrategyStandardImpl");
-
-        return application
-                .sources(NiFiRegistryApiTestApplication.class)
-                .properties(fixedProps);
-    }
-
-    public static void main(String[] args) {
-        SpringApplication.run(NiFiRegistryApiTestApplication.class, args);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/NiFiRegistryTestApiApplication.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/NiFiRegistryTestApiApplication.java b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/NiFiRegistryTestApiApplication.java
new file mode 100644
index 0000000..e55fb21
--- /dev/null
+++ b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/NiFiRegistryTestApiApplication.java
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+import org.apache.nifi.registry.db.DataSourceFactory;
+import org.springframework.boot.autoconfigure.SpringBootApplication;
+import org.springframework.boot.web.support.SpringBootServletInitializer;
+import org.springframework.context.annotation.ComponentScan;
+import org.springframework.context.annotation.FilterType;
+
+@SpringBootApplication
+@ComponentScan(
+        excludeFilters = {
+                @ComponentScan.Filter(
+                        type = FilterType.ASSIGNABLE_TYPE,
+                        value = SpringBootServletInitializer.class), // Avoid loading NiFiRegistryApiApplication
+                @ComponentScan.Filter(
+                        type = FilterType.ASSIGNABLE_TYPE,
+                        value = DataSourceFactory.class), // Avoid loading DataSourceFactory
+                @ComponentScan.Filter(
+                        type = FilterType.REGEX,
+                        pattern = "org\\.apache\\.nifi\\.registry\\.NiFiRegistryPropertiesFactory"), // Avoid loading NiFiRegistryPropertiesFactory
+        })
+public class NiFiRegistryTestApiApplication extends SpringBootServletInitializer {
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/SecureLdapTestApiApplication.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/SecureLdapTestApiApplication.java b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/SecureLdapTestApiApplication.java
new file mode 100644
index 0000000..fea8349
--- /dev/null
+++ b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/SecureLdapTestApiApplication.java
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+import org.apache.nifi.registry.db.DataSourceFactory;
+import org.apache.nifi.registry.security.authorization.AuthorizerFactory;
+import org.springframework.boot.autoconfigure.SpringBootApplication;
+import org.springframework.boot.web.support.SpringBootServletInitializer;
+import org.springframework.context.annotation.ComponentScan;
+import org.springframework.context.annotation.FilterType;
+
+@SpringBootApplication
+@ComponentScan(
+        basePackages = "org.apache.nifi.registry",
+        excludeFilters = {
+                @ComponentScan.Filter(
+                        type = FilterType.ASSIGNABLE_TYPE,
+                        value = SpringBootServletInitializer.class), // Avoid loading NiFiRegistryApiApplication
+                @ComponentScan.Filter(
+                        type = FilterType.ASSIGNABLE_TYPE,
+                        value = DataSourceFactory.class), // Avoid loading DataSourceFactory
+                @ComponentScan.Filter(
+                        type = FilterType.ASSIGNABLE_TYPE,
+                        value = AuthorizerFactory.class), // Avoid loading AuthorizerFactory.getAuthorizer(), as we need to add it again with test-specific @DependsOn annotation
+                @ComponentScan.Filter(
+                        type = FilterType.REGEX,
+                        pattern = "org\\.apache\\.nifi\\.registry\\.NiFiRegistryPropertiesFactory"), // Avoid loading NiFiRegistryPropertiesFactory
+        })
+public class SecureLdapTestApiApplication extends SpringBootServletInitializer {
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureFileIT.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureFileIT.java b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureFileIT.java
index d04e79b..2ca3caa 100644
--- a/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureFileIT.java
+++ b/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureFileIT.java
@@ -16,7 +16,7 @@
  */
 package org.apache.nifi.registry.web.api;
 
-import org.apache.nifi.registry.NiFiRegistryApiTestApplication;
+import org.apache.nifi.registry.NiFiRegistryTestApiApplication;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.skyscreamer.jsonassert.JSONAssert;
@@ -39,7 +39,7 @@ import static org.junit.Assert.assertEquals;
  */
 @RunWith(SpringRunner.class)
 @SpringBootTest(
-        classes = NiFiRegistryApiTestApplication.class,
+        classes = NiFiRegistryTestApiApplication.class,
         webEnvironment = SpringBootTest.WebEnvironment.RANDOM_PORT,
         properties = "spring.profiles.include=ITSecureFile")
 @Import(SecureITClientConfiguration.class)


[11/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/StandardNiFiUser.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/StandardNiFiUser.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/StandardNiFiUser.java
new file mode 100644
index 0000000..92c2274
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/StandardNiFiUser.java
@@ -0,0 +1,189 @@
+/*
+ * 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.authorization.user;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.util.Collections;
+import java.util.Objects;
+import java.util.Set;
+
+/**
+ * An implementation of NiFiUser.
+ */
+public class StandardNiFiUser implements NiFiUser {
+
+    public static final String ANONYMOUS_IDENTITY = "anonymous";
+    public static final StandardNiFiUser ANONYMOUS = new Builder().identity(ANONYMOUS_IDENTITY).anonymous(true).build();
+
+    private final String identity;
+    private final Set<String> groups;
+    private final NiFiUser chain;
+    private final String clientAddress;
+    private final boolean isAnonymous;
+
+    private StandardNiFiUser(final Builder builder) {
+        this.identity = builder.identity;
+        this.groups = builder.groups == null ? null : Collections.unmodifiableSet(builder.groups);
+        this.chain = builder.chain;
+        this.clientAddress = builder.clientAddress;
+        this.isAnonymous = builder.isAnonymous;
+    }
+
+    /**
+     * This static builder allows the chain and clientAddress to be populated without allowing calling code to provide a non-anonymous identity of the anonymous user.
+     *
+     * @param chain the proxied entities in {@see NiFiUser} form
+     * @param clientAddress the address the request originated from
+     * @return an anonymous user instance with the identity "anonymous"
+     */
+    public static StandardNiFiUser populateAnonymousUser(NiFiUser chain, String clientAddress) {
+        return new Builder().identity(ANONYMOUS_IDENTITY).chain(chain).clientAddress(clientAddress).anonymous(true).build();
+    }
+
+    @Override
+    public String getIdentity() {
+        return identity;
+    }
+
+    @Override
+    public Set<String> getGroups() {
+        return groups;
+    }
+
+    @Override
+    public NiFiUser getChain() {
+        return chain;
+    }
+
+    @Override
+    public boolean isAnonymous() {
+        return isAnonymous;
+    }
+
+    @Override
+    public String getClientAddress() {
+        return clientAddress;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == null) {
+            return false;
+        }
+
+        if (!(obj instanceof NiFiUser)) {
+            return false;
+        }
+
+        final NiFiUser other = (NiFiUser) obj;
+        return Objects.equals(this.identity, other.getIdentity());
+    }
+
+    @Override
+    public int hashCode() {
+        int hash = 7;
+        hash = 53 * hash + Objects.hashCode(this.identity);
+        return hash;
+    }
+
+    @Override
+    public String toString() {
+        final String formattedGroups;
+        if (groups == null) {
+            formattedGroups = "none";
+        } else {
+            formattedGroups = StringUtils.join(groups, ", ");
+        }
+
+        return String.format("identity[%s], groups[%s]", getIdentity(), formattedGroups);
+    }
+
+    /**
+     * Builder for a StandardNiFiUser
+     */
+    public static class Builder {
+
+        private String identity;
+        private Set<String> groups;
+        private NiFiUser chain;
+        private String clientAddress;
+        private boolean isAnonymous = false;
+
+        /**
+         * Sets the identity.
+         *
+         * @param identity the identity string for the user (i.e. "Andy" or "CN=alopresto, OU=Apache NiFi")
+         * @return the builder
+         */
+        public Builder identity(final String identity) {
+            this.identity = identity;
+            return this;
+        }
+
+        /**
+         * Sets the groups.
+         *
+         * @param groups the user groups
+         * @return the builder
+         */
+        public Builder groups(final Set<String> groups) {
+            this.groups = groups;
+            return this;
+        }
+
+        /**
+         * Sets the chain.
+         *
+         * @param chain the proxy chain that leads to this users
+         * @return the builder
+         */
+        public Builder chain(final NiFiUser chain) {
+            this.chain = chain;
+            return this;
+        }
+
+        /**
+         * Sets the client address.
+         *
+         * @param clientAddress the source address of the request
+         * @return the builder
+         */
+        public Builder clientAddress(final String clientAddress) {
+            this.clientAddress = clientAddress;
+            return this;
+        }
+
+        /**
+         * Sets whether this user is the canonical "anonymous" user
+         *
+         * @param isAnonymous true to represent the canonical "anonymous" user
+         * @return the builder
+         */
+        private Builder anonymous(final boolean isAnonymous) {
+            this.isAnonymous = isAnonymous;
+            return this;
+        }
+
+        /**
+         * @return builds a StandardNiFiUser from the current state of the builder
+         */
+        public StandardNiFiUser build() {
+            return new StandardNiFiUser(this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/key/Key.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/key/Key.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/key/Key.java
new file mode 100644
index 0000000..c110fa8
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/key/Key.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.security.key;
+
+import java.io.Serializable;
+
+/**
+ * An signing key for a NiFi user.
+ */
+public class Key implements Serializable {
+
+    private String id;
+    private String identity;
+    private String key;
+
+    /**
+     * The key id.
+     *
+     * @return the id
+     */
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    /**
+     * The identity of the user this key is associated with.
+     *
+     * @return the identity
+     */
+    public String getIdentity() {
+        return identity;
+    }
+
+    public void setIdentity(String identity) {
+        this.identity = identity;
+    }
+
+    /**
+     * The signing key.
+     *
+     * @return the signing key
+     */
+    public String getKey() {
+        return key;
+    }
+
+    public void setKey(String key) {
+        this.key = key;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/key/KeyService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/key/KeyService.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/key/KeyService.java
new file mode 100644
index 0000000..3b9a7ca
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/key/KeyService.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.security.key;
+
+/**
+ * Manages NiFi user keys.
+ */
+public interface KeyService {
+
+    /**
+     * Gets a key for the specified user identity. Returns null if the user has not had a key issued
+     *
+     * @param id The key id
+     * @return The key or null
+     */
+    Key getKey(String id);
+
+    /**
+     * Gets a key for the specified user identity. If a key does not exist, one will be created.
+     *
+     * @param identity The user identity
+     * @return The key
+     */
+    Key getOrCreateKey(String identity);
+
+    /**
+     * Deletes keys for the specified identity.
+     *
+     * @param identity The user identity
+     */
+    void deleteKey(String identity);
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/IdentityStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/IdentityStrategy.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/IdentityStrategy.java
new file mode 100644
index 0000000..135f261
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/IdentityStrategy.java
@@ -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.
+ */
+package org.apache.nifi.registry.security.ldap;
+
+public enum IdentityStrategy {
+    USE_DN,
+    USE_USERNAME;
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapAuthenticationStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapAuthenticationStrategy.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapAuthenticationStrategy.java
new file mode 100644
index 0000000..331fbc3
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapAuthenticationStrategy.java
@@ -0,0 +1,24 @@
+/*
+ * 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.ldap;
+
+public enum LdapAuthenticationStrategy {
+    ANONYMOUS,
+    SIMPLE,
+    LDAPS,
+    START_TLS
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/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
new file mode 100644
index 0000000..6beebc5
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapIdentityProvider.java
@@ -0,0 +1,348 @@
+/*
+ * 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.ldap;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.security.authentication.AuthenticationResponse;
+import org.apache.nifi.registry.security.authentication.LoginCredentials;
+import org.apache.nifi.registry.security.authentication.LoginIdentityProvider;
+import org.apache.nifi.registry.security.authentication.LoginIdentityProviderConfigurationContext;
+import org.apache.nifi.registry.security.authentication.LoginIdentityProviderInitializationContext;
+import org.apache.nifi.registry.security.authentication.exception.IdentityAccessException;
+import org.apache.nifi.registry.security.authentication.exception.InvalidLoginCredentialsException;
+import org.apache.nifi.registry.security.authentication.exception.ProviderCreationException;
+import org.apache.nifi.registry.security.authentication.exception.ProviderDestructionException;
+import org.apache.nifi.registry.security.util.SslContextFactory;
+import org.apache.nifi.registry.security.util.SslContextFactory.ClientAuth;
+import org.apache.nifi.registry.util.FormatUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.ldap.AuthenticationException;
+import org.springframework.ldap.core.support.AbstractTlsDirContextAuthenticationStrategy;
+import org.springframework.ldap.core.support.DefaultTlsDirContextAuthenticationStrategy;
+import org.springframework.ldap.core.support.LdapContextSource;
+import org.springframework.ldap.core.support.SimpleDirContextAuthenticationStrategy;
+import org.springframework.security.authentication.BadCredentialsException;
+import org.springframework.security.authentication.UsernamePasswordAuthenticationToken;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.userdetails.UsernameNotFoundException;
+import org.springframework.security.ldap.authentication.AbstractLdapAuthenticationProvider;
+import org.springframework.security.ldap.authentication.BindAuthenticator;
+import org.springframework.security.ldap.authentication.LdapAuthenticationProvider;
+import org.springframework.security.ldap.search.FilterBasedLdapUserSearch;
+import org.springframework.security.ldap.search.LdapUserSearch;
+import org.springframework.security.ldap.userdetails.LdapUserDetails;
+
+import javax.naming.Context;
+import javax.net.ssl.SSLContext;
+import java.io.IOException;
+import java.security.KeyManagementException;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Abstract LDAP based implementation of a login identity provider.
+ */
+public class LdapIdentityProvider implements LoginIdentityProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(LdapIdentityProvider.class);
+
+    private AbstractLdapAuthenticationProvider provider;
+    private String issuer;
+    private long expiration;
+    private IdentityStrategy identityStrategy;
+
+    @Override
+    public final void initialize(final LoginIdentityProviderInitializationContext initializationContext) throws ProviderCreationException {
+        this.issuer = getClass().getSimpleName();
+    }
+
+    @Override
+    public final void onConfigured(final LoginIdentityProviderConfigurationContext configurationContext) throws ProviderCreationException {
+        final String rawExpiration = configurationContext.getProperty("Authentication Expiration");
+        if (StringUtils.isBlank(rawExpiration)) {
+            throw new ProviderCreationException("The Authentication Expiration must be specified.");
+        }
+
+        try {
+            expiration = FormatUtils.getTimeDuration(rawExpiration, TimeUnit.MILLISECONDS);
+        } catch (final IllegalArgumentException iae) {
+            throw new ProviderCreationException(String.format("The Expiration Duration '%s' is not a valid time duration", rawExpiration));
+        }
+
+        final LdapContextSource context = new LdapContextSource();
+
+        final Map<String, Object> baseEnvironment = new HashMap<>();
+
+        // connect/read time out
+        setTimeout(configurationContext, baseEnvironment, "Connect Timeout", "com.sun.jndi.ldap.connect.timeout");
+        setTimeout(configurationContext, baseEnvironment, "Read Timeout", "com.sun.jndi.ldap.read.timeout");
+
+        // authentication strategy
+        final String rawAuthenticationStrategy = configurationContext.getProperty("Authentication Strategy");
+        final LdapAuthenticationStrategy authenticationStrategy;
+        try {
+            authenticationStrategy = LdapAuthenticationStrategy.valueOf(rawAuthenticationStrategy);
+        } catch (final IllegalArgumentException iae) {
+            throw new ProviderCreationException(String.format("Unrecognized authentication strategy '%s'. Possible values are [%s]",
+                    rawAuthenticationStrategy, StringUtils.join(LdapAuthenticationStrategy.values(), ", ")));
+        }
+
+        switch (authenticationStrategy) {
+            case ANONYMOUS:
+                context.setAnonymousReadOnly(true);
+                break;
+            default:
+                final String userDn = configurationContext.getProperty("Manager DN");
+                final String password = configurationContext.getProperty("Manager Password");
+
+                context.setUserDn(userDn);
+                context.setPassword(password);
+
+                switch (authenticationStrategy) {
+                    case SIMPLE:
+                        context.setAuthenticationStrategy(new SimpleDirContextAuthenticationStrategy());
+                        break;
+                    case LDAPS:
+                        context.setAuthenticationStrategy(new SimpleDirContextAuthenticationStrategy());
+
+                        // indicate a secure connection
+                        baseEnvironment.put(Context.SECURITY_PROTOCOL, "ssl");
+
+                        // get the configured ssl context
+                        final SSLContext ldapsSslContext = getConfiguredSslContext(configurationContext);
+                        if (ldapsSslContext != null) {
+                            // initialize the ldaps socket factory prior to use
+                            LdapsSocketFactory.initialize(ldapsSslContext.getSocketFactory());
+                            baseEnvironment.put("java.naming.ldap.factory.socket", LdapsSocketFactory.class.getName());
+                        }
+                        break;
+                    case START_TLS:
+                        final AbstractTlsDirContextAuthenticationStrategy tlsAuthenticationStrategy = new DefaultTlsDirContextAuthenticationStrategy();
+
+                        // shutdown gracefully
+                        final String rawShutdownGracefully = configurationContext.getProperty("TLS - Shutdown Gracefully");
+                        if (StringUtils.isNotBlank(rawShutdownGracefully)) {
+                            final boolean shutdownGracefully = Boolean.TRUE.toString().equalsIgnoreCase(rawShutdownGracefully);
+                            tlsAuthenticationStrategy.setShutdownTlsGracefully(shutdownGracefully);
+                        }
+
+                        // get the configured ssl context
+                        final SSLContext startTlsSslContext = getConfiguredSslContext(configurationContext);
+                        if (startTlsSslContext != null) {
+                            tlsAuthenticationStrategy.setSslSocketFactory(startTlsSslContext.getSocketFactory());
+                        }
+
+                        // set the authentication strategy
+                        context.setAuthenticationStrategy(tlsAuthenticationStrategy);
+                        break;
+                }
+                break;
+        }
+
+        // referrals
+        final String rawReferralStrategy = configurationContext.getProperty("Referral Strategy");
+
+        final ReferralStrategy referralStrategy;
+        try {
+            referralStrategy = ReferralStrategy.valueOf(rawReferralStrategy);
+        } catch (final IllegalArgumentException iae) {
+            throw new ProviderCreationException(String.format("Unrecognized referral strategy '%s'. Possible values are [%s]",
+                    rawReferralStrategy, StringUtils.join(ReferralStrategy.values(), ", ")));
+        }
+
+        // using the value as this needs to be the lowercase version while the value is configured with the enum constant
+        context.setReferral(referralStrategy.getValue());
+
+        // url
+        final String urls = configurationContext.getProperty("Url");
+
+        if (StringUtils.isBlank(urls)) {
+            throw new ProviderCreationException("LDAP identity provider 'Url' must be specified.");
+        }
+
+        // connection
+        context.setUrls(StringUtils.split(urls));
+
+        // search criteria
+        final String userSearchBase = configurationContext.getProperty("User Search Base");
+        final String userSearchFilter = configurationContext.getProperty("User Search Filter");
+
+        if (StringUtils.isBlank(userSearchBase) || StringUtils.isBlank(userSearchFilter)) {
+            throw new ProviderCreationException("LDAP identity provider 'User Search Base' and 'User Search Filter' must be specified.");
+        }
+
+        final LdapUserSearch userSearch = new FilterBasedLdapUserSearch(userSearchBase, userSearchFilter, context);
+
+        // bind
+        final BindAuthenticator authenticator = new BindAuthenticator(context);
+        authenticator.setUserSearch(userSearch);
+
+        // identity strategy
+        final String rawIdentityStrategy = configurationContext.getProperty("Identity Strategy");
+
+        if (StringUtils.isBlank(rawIdentityStrategy)) {
+            logger.info(String.format("Identity Strategy is not configured, defaulting strategy to %s.", IdentityStrategy.USE_DN));
+
+            // if this value is not configured, default to use dn which was the previous implementation
+            identityStrategy = IdentityStrategy.USE_DN;
+        } else {
+            try {
+                // attempt to get the configured identity strategy
+                identityStrategy = IdentityStrategy.valueOf(rawIdentityStrategy);
+            } catch (final IllegalArgumentException iae) {
+                throw new ProviderCreationException(String.format("Unrecognized identity strategy '%s'. Possible values are [%s]",
+                        rawIdentityStrategy, StringUtils.join(IdentityStrategy.values(), ", ")));
+            }
+        }
+
+        // set the base environment is necessary
+        if (!baseEnvironment.isEmpty()) {
+            context.setBaseEnvironmentProperties(baseEnvironment);
+        }
+
+        try {
+            // handling initializing beans
+            context.afterPropertiesSet();
+            authenticator.afterPropertiesSet();
+        } catch (final Exception e) {
+            throw new ProviderCreationException(e.getMessage(), e);
+        }
+
+        // create the underlying provider
+        provider = new LdapAuthenticationProvider(authenticator);
+    }
+
+    private void setTimeout(final LoginIdentityProviderConfigurationContext configurationContext,
+            final Map<String, Object> baseEnvironment,
+            final String configurationProperty,
+            final String environmentKey) {
+
+        final String rawTimeout = configurationContext.getProperty(configurationProperty);
+        if (StringUtils.isNotBlank(rawTimeout)) {
+            try {
+                final Long timeout = FormatUtils.getTimeDuration(rawTimeout, TimeUnit.MILLISECONDS);
+                baseEnvironment.put(environmentKey, timeout.toString());
+            } catch (final IllegalArgumentException iae) {
+                throw new ProviderCreationException(String.format("The %s '%s' is not a valid time duration", configurationProperty, rawTimeout));
+            }
+        }
+    }
+
+    private SSLContext getConfiguredSslContext(final LoginIdentityProviderConfigurationContext configurationContext) {
+        final String rawKeystore = configurationContext.getProperty("TLS - Keystore");
+        final String rawKeystorePassword = configurationContext.getProperty("TLS - Keystore Password");
+        final String rawKeystoreType = configurationContext.getProperty("TLS - Keystore Type");
+        final String rawTruststore = configurationContext.getProperty("TLS - Truststore");
+        final String rawTruststorePassword = configurationContext.getProperty("TLS - Truststore Password");
+        final String rawTruststoreType = configurationContext.getProperty("TLS - Truststore Type");
+        final String rawClientAuth = configurationContext.getProperty("TLS - Client Auth");
+        final String rawProtocol = configurationContext.getProperty("TLS - Protocol");
+
+        // create the ssl context
+        final SSLContext sslContext;
+        try {
+            if (StringUtils.isBlank(rawKeystore) && StringUtils.isBlank(rawTruststore)) {
+                sslContext = null;
+            } else {
+                // ensure the protocol is specified
+                if (StringUtils.isBlank(rawProtocol)) {
+                    throw new ProviderCreationException("TLS - Protocol must be specified.");
+                }
+
+                if (StringUtils.isBlank(rawKeystore)) {
+                    sslContext = SslContextFactory.createTrustSslContext(rawTruststore, rawTruststorePassword.toCharArray(), rawTruststoreType, rawProtocol);
+                } else if (StringUtils.isBlank(rawTruststore)) {
+                    sslContext = SslContextFactory.createSslContext(rawKeystore, rawKeystorePassword.toCharArray(), rawKeystoreType, rawProtocol);
+                } else {
+                    // determine the client auth if specified
+                    final ClientAuth clientAuth;
+                    if (StringUtils.isBlank(rawClientAuth)) {
+                        clientAuth = ClientAuth.NONE;
+                    } else {
+                        try {
+                            clientAuth = ClientAuth.valueOf(rawClientAuth);
+                        } catch (final IllegalArgumentException iae) {
+                            throw new ProviderCreationException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
+                                    rawClientAuth, StringUtils.join(ClientAuth.values(), ", ")));
+                        }
+                    }
+
+                    sslContext = SslContextFactory.createSslContext(rawKeystore, rawKeystorePassword.toCharArray(), rawKeystoreType,
+                            rawTruststore, rawTruststorePassword.toCharArray(), rawTruststoreType, clientAuth, rawProtocol);
+                }
+            }
+        } catch (final KeyStoreException | NoSuchAlgorithmException | CertificateException | UnrecoverableKeyException | KeyManagementException | IOException e) {
+            throw new ProviderCreationException(e.getMessage(), e);
+        }
+
+        return sslContext;
+    }
+
+    @Override
+    public final AuthenticationResponse authenticate(final LoginCredentials credentials) throws InvalidLoginCredentialsException, IdentityAccessException {
+        if (provider == null) {
+            throw new IdentityAccessException("The LDAP authentication provider is not initialized.");
+        }
+
+        try {
+            // perform the authentication
+            final UsernamePasswordAuthenticationToken token = new UsernamePasswordAuthenticationToken(credentials.getUsername(), credentials.getPassword());
+            final Authentication authentication = provider.authenticate(token);
+
+            // use dn if configured
+            if (IdentityStrategy.USE_DN.equals(identityStrategy)) {
+                // attempt to get the ldap user details to get the DN
+                if (authentication.getPrincipal() instanceof LdapUserDetails) {
+                    final LdapUserDetails userDetails = (LdapUserDetails) authentication.getPrincipal();
+                    return new AuthenticationResponse(userDetails.getDn(), credentials.getUsername(), expiration, issuer);
+                } else {
+                    logger.warn(String.format("Unable to determine user DN for %s, using username.", authentication.getName()));
+                    return new AuthenticationResponse(authentication.getName(), credentials.getUsername(), expiration, issuer);
+                }
+            } else {
+                return new AuthenticationResponse(authentication.getName(), credentials.getUsername(), expiration, issuer);
+            }
+        } catch (final BadCredentialsException | UsernameNotFoundException | AuthenticationException e) {
+            throw new InvalidLoginCredentialsException(e.getMessage(), e);
+        } catch (final Exception e) {
+            // there appears to be a bug that generates a InternalAuthenticationServiceException wrapped around an AuthenticationException. this
+            // shouldn't be the case as they the service exception suggestions that something was wrong with the service. while the authentication
+            // exception suggests that username and/or credentials were incorrect. checking the cause seems to address this scenario.
+            final Throwable cause = e.getCause();
+            if (cause instanceof AuthenticationException) {
+                throw new InvalidLoginCredentialsException(e.getMessage(), e);
+            }
+
+            logger.error(e.getMessage());
+            if (logger.isDebugEnabled()) {
+                logger.debug(StringUtils.EMPTY, e);
+            }
+            throw new IdentityAccessException("Unable to validate the supplied credentials. Please contact the system administrator.", e);
+        }
+    }
+
+    @Override
+    public final void preDestruction() throws ProviderDestructionException {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapsSocketFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapsSocketFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapsSocketFactory.java
new file mode 100644
index 0000000..dff9572
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/LdapsSocketFactory.java
@@ -0,0 +1,106 @@
+/*
+ * 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.ldap;
+
+import javax.net.SocketFactory;
+import javax.net.ssl.SSLSocketFactory;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.UnknownHostException;
+
+/**
+ * SSLSocketFactory used when connecting to a Directory Server over LDAPS.
+ */
+public class LdapsSocketFactory extends SSLSocketFactory {
+
+    // singleton
+    private static LdapsSocketFactory instance;
+
+    // delegate
+    private SSLSocketFactory delegate;
+
+    /**
+     * Initializes the LdapsSocketFactory with the specified SSLSocketFactory. The specified
+     * socket factory will be used as a delegate for all subsequent instances of this class.
+     *
+     * @param sslSocketFactory delegate socket factory
+     */
+    public static void initialize(final SSLSocketFactory sslSocketFactory) {
+        instance = new LdapsSocketFactory(sslSocketFactory);
+    }
+
+    /**
+     * Gets the LdapsSocketFactory that was previously initialized.
+     *
+      * @return socket factory
+     */
+    public static SocketFactory getDefault() {
+        return instance;
+    }
+
+    /**
+     * Creates a new LdapsSocketFactory.
+     *
+     * @param sslSocketFactory delegate socket factory
+     */
+    private LdapsSocketFactory(final SSLSocketFactory sslSocketFactory) {
+        delegate = sslSocketFactory;
+    }
+
+    // delegate methods
+
+    @Override
+    public String[] getSupportedCipherSuites() {
+        return delegate.getSupportedCipherSuites();
+    }
+
+    @Override
+    public String[] getDefaultCipherSuites() {
+        return delegate.getDefaultCipherSuites();
+    }
+
+    @Override
+    public Socket createSocket(Socket socket, String string, int i, boolean bln) throws IOException {
+        return delegate.createSocket(socket, string, i, bln);
+    }
+
+    @Override
+    public Socket createSocket(InetAddress ia, int i, InetAddress ia1, int i1) throws IOException {
+        return delegate.createSocket(ia, i, ia1, i1);
+    }
+
+    @Override
+    public Socket createSocket(InetAddress ia, int i) throws IOException {
+        return delegate.createSocket(ia, i);
+    }
+
+    @Override
+    public Socket createSocket(String string, int i, InetAddress ia, int i1) throws IOException, UnknownHostException {
+        return delegate.createSocket(string, i, ia, i1);
+    }
+
+    @Override
+    public Socket createSocket(String string, int i) throws IOException, UnknownHostException {
+        return delegate.createSocket(string, i);
+    }
+
+    @Override
+    public Socket createSocket() throws IOException {
+        return delegate.createSocket();
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/ReferralStrategy.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/ReferralStrategy.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/ReferralStrategy.java
new file mode 100644
index 0000000..4258cde
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/ReferralStrategy.java
@@ -0,0 +1,35 @@
+/*
+ * 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.ldap;
+
+public enum ReferralStrategy {
+
+    FOLLOW("follow"),
+    IGNORE("ignore"),
+    THROW("throw");
+
+    private final String value;
+
+    private ReferralStrategy(String value) {
+        this.value = value;
+    }
+
+    public String getValue() {
+        return value;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/LdapUserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/LdapUserGroupProvider.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/LdapUserGroupProvider.java
new file mode 100644
index 0000000..af10ece
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/LdapUserGroupProvider.java
@@ -0,0 +1,750 @@
+/*
+ * 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.ldap.tenants;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.properties.util.IdentityMapping;
+import org.apache.nifi.registry.properties.util.IdentityMappingUtil;
+import org.apache.nifi.registry.security.authentication.exception.ProviderDestructionException;
+import org.apache.nifi.registry.security.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.User;
+import org.apache.nifi.registry.security.authorization.UserAndGroups;
+import org.apache.nifi.registry.security.authorization.UserGroupProvider;
+import org.apache.nifi.registry.security.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.registry.security.authorization.annotation.AuthorizerContext;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.ldap.LdapAuthenticationStrategy;
+import org.apache.nifi.registry.security.ldap.LdapsSocketFactory;
+import org.apache.nifi.registry.security.ldap.ReferralStrategy;
+import org.apache.nifi.registry.security.util.SslContextFactory;
+import org.apache.nifi.registry.security.util.SslContextFactory.ClientAuth;
+import org.apache.nifi.registry.util.FormatUtils;
+import org.apache.nifi.registry.util.PropertyValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.ldap.control.PagedResultsDirContextProcessor;
+import org.springframework.ldap.core.ContextSource;
+import org.springframework.ldap.core.DirContextAdapter;
+import org.springframework.ldap.core.DirContextOperations;
+import org.springframework.ldap.core.DirContextProcessor;
+import org.springframework.ldap.core.LdapTemplate;
+import org.springframework.ldap.core.LdapTemplate.NullDirContextProcessor;
+import org.springframework.ldap.core.support.AbstractContextMapper;
+import org.springframework.ldap.core.support.AbstractTlsDirContextAuthenticationStrategy;
+import org.springframework.ldap.core.support.DefaultTlsDirContextAuthenticationStrategy;
+import org.springframework.ldap.core.support.LdapContextSource;
+import org.springframework.ldap.core.support.SimpleDirContextAuthenticationStrategy;
+import org.springframework.ldap.core.support.SingleContextSource;
+import org.springframework.ldap.filter.AndFilter;
+import org.springframework.ldap.filter.EqualsFilter;
+import org.springframework.ldap.filter.HardcodedFilter;
+
+import javax.naming.Context;
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.directory.Attribute;
+import javax.naming.directory.SearchControls;
+import javax.net.ssl.SSLContext;
+import java.io.IOException;
+import java.security.KeyManagementException;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Abstract LDAP based implementation of a login identity provider.
+ */
+public class LdapUserGroupProvider implements UserGroupProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(LdapUserGroupProvider.class);
+
+    public static final String PROP_CONNECT_TIMEOUT = "Connect Timeout";
+    public static final String PROP_READ_TIMEOUT = "Read Timeout";
+    public static final String PROP_AUTHENTICATION_STRATEGY = "Authentication Strategy";
+    public static final String PROP_MANAGER_DN = "Manager DN";
+    public static final String PROP_MANAGER_PASSWORD = "Manager Password";
+    public static final String PROP_REFERRAL_STRATEGY = "Referral Strategy";
+    public static final String PROP_URL = "Url";
+    public static final String PROP_PAGE_SIZE = "Page Size";
+
+    public static final String PROP_USER_SEARCH_BASE = "User Search Base";
+    public static final String PROP_USER_OBJECT_CLASS = "User Object Class";
+    public static final String PROP_USER_SEARCH_SCOPE = "User Search Scope";
+    public static final String PROP_USER_SEARCH_FILTER = "User Search Filter";
+    public static final String PROP_USER_IDENTITY_ATTRIBUTE = "User Identity Attribute";
+    public static final String PROP_USER_GROUP_ATTRIBUTE = "User Group Name Attribute";
+
+    public static final String PROP_GROUP_SEARCH_BASE = "Group Search Base";
+    public static final String PROP_GROUP_OBJECT_CLASS = "Group Object Class";
+    public static final String PROP_GROUP_SEARCH_SCOPE = "Group Search Scope";
+    public static final String PROP_GROUP_SEARCH_FILTER = "Group Search Filter";
+    public static final String PROP_GROUP_NAME_ATTRIBUTE = "Group Name Attribute";
+    public static final String PROP_GROUP_MEMBER_ATTRIBUTE = "Group Member Attribute";
+
+    public static final String PROP_SYNC_INTERVAL = "Sync Interval";
+
+    //private AuthorizerConfigurationContext configurationContext;
+
+    private List<IdentityMapping> identityMappings;
+    private NiFiRegistryProperties properties;
+
+    private ScheduledExecutorService ldapSync;
+    private AtomicReference<TenantHolder> tenants = new AtomicReference<>(null);
+
+    private String userSearchBase;
+    private SearchScope userSearchScope;
+    private String userSearchFilter;
+    private String userIdentityAttribute;
+    private String userObjectClass;
+    private String userGroupNameAttribute;
+    private boolean useDnForUserIdentity;
+    private boolean performUserSearch;
+
+    private String groupSearchBase;
+    private SearchScope groupSearchScope;
+    private String groupSearchFilter;
+    private String groupMemberAttribute;
+    private String groupNameAttribute;
+    private String groupObjectClass;
+    private boolean useDnForGroupName;
+    private boolean performGroupSearch;
+
+    private Integer pageSize;
+
+    @Override
+    public void initialize(final UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
+        ldapSync = Executors.newSingleThreadScheduledExecutor(new ThreadFactory() {
+            final ThreadFactory factory = Executors.defaultThreadFactory();
+
+            @Override
+            public Thread newThread(Runnable r) {
+                final Thread thread = factory.newThread(r);
+                thread.setName(String.format("%s (%s) - background sync thread", getClass().getSimpleName(), initializationContext.getIdentifier()));
+                return thread;
+            }
+        });
+    }
+
+    @Override
+    public void onConfigured(final AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        final LdapContextSource context = new LdapContextSource();
+
+        final Map<String, Object> baseEnvironment = new HashMap<>();
+
+        // connect/read time out
+        setTimeout(configurationContext, baseEnvironment, PROP_CONNECT_TIMEOUT, "com.sun.jndi.ldap.connect.timeout");
+        setTimeout(configurationContext, baseEnvironment, PROP_READ_TIMEOUT, "com.sun.jndi.ldap.read.timeout");
+
+        // authentication strategy
+        final PropertyValue rawAuthenticationStrategy = configurationContext.getProperty(PROP_AUTHENTICATION_STRATEGY);
+        final LdapAuthenticationStrategy authenticationStrategy;
+        try {
+            authenticationStrategy = LdapAuthenticationStrategy.valueOf(rawAuthenticationStrategy.getValue());
+        } catch (final IllegalArgumentException iae) {
+            throw new AuthorizerCreationException(String.format("Unrecognized authentication strategy '%s'. Possible values are [%s]",
+                    rawAuthenticationStrategy.getValue(), StringUtils.join(LdapAuthenticationStrategy.values(), ", ")));
+        }
+
+        switch (authenticationStrategy) {
+            case ANONYMOUS:
+                context.setAnonymousReadOnly(true);
+                break;
+            default:
+                final String userDn = configurationContext.getProperty(PROP_MANAGER_DN).getValue();
+                final String password = configurationContext.getProperty(PROP_MANAGER_PASSWORD).getValue();
+
+                context.setUserDn(userDn);
+                context.setPassword(password);
+
+                switch (authenticationStrategy) {
+                    case SIMPLE:
+                        context.setAuthenticationStrategy(new SimpleDirContextAuthenticationStrategy());
+                        break;
+                    case LDAPS:
+                        context.setAuthenticationStrategy(new SimpleDirContextAuthenticationStrategy());
+
+                        // indicate a secure connection
+                        baseEnvironment.put(Context.SECURITY_PROTOCOL, "ssl");
+
+                        // get the configured ssl context
+                        final SSLContext ldapsSslContext = getConfiguredSslContext(configurationContext);
+                        if (ldapsSslContext != null) {
+                            // initialize the ldaps socket factory prior to use
+                            LdapsSocketFactory.initialize(ldapsSslContext.getSocketFactory());
+                            baseEnvironment.put("java.naming.ldap.factory.socket", LdapsSocketFactory.class.getName());
+                        }
+                        break;
+                    case START_TLS:
+                        final AbstractTlsDirContextAuthenticationStrategy tlsAuthenticationStrategy = new DefaultTlsDirContextAuthenticationStrategy();
+
+                        // shutdown gracefully
+                        final String rawShutdownGracefully = configurationContext.getProperty("TLS - Shutdown Gracefully").getValue();
+                        if (StringUtils.isNotBlank(rawShutdownGracefully)) {
+                            final boolean shutdownGracefully = Boolean.TRUE.toString().equalsIgnoreCase(rawShutdownGracefully);
+                            tlsAuthenticationStrategy.setShutdownTlsGracefully(shutdownGracefully);
+                        }
+
+                        // get the configured ssl context
+                        final SSLContext startTlsSslContext = getConfiguredSslContext(configurationContext);
+                        if (startTlsSslContext != null) {
+                            tlsAuthenticationStrategy.setSslSocketFactory(startTlsSslContext.getSocketFactory());
+                        }
+
+                        // set the authentication strategy
+                        context.setAuthenticationStrategy(tlsAuthenticationStrategy);
+                        break;
+                }
+                break;
+        }
+
+        // referrals
+        final String rawReferralStrategy = configurationContext.getProperty(PROP_REFERRAL_STRATEGY).getValue();
+
+        final ReferralStrategy referralStrategy;
+        try {
+            referralStrategy = ReferralStrategy.valueOf(rawReferralStrategy);
+        } catch (final IllegalArgumentException iae) {
+            throw new AuthorizerCreationException(String.format("Unrecognized referral strategy '%s'. Possible values are [%s]",
+                    rawReferralStrategy, StringUtils.join(ReferralStrategy.values(), ", ")));
+        }
+
+        // using the value as this needs to be the lowercase version while the value is configured with the enum constant
+        context.setReferral(referralStrategy.getValue());
+
+        // url
+        final String urls = configurationContext.getProperty(PROP_URL).getValue();
+
+        if (StringUtils.isBlank(urls)) {
+            throw new AuthorizerCreationException("LDAP identity provider 'Url' must be specified.");
+        }
+
+        // connection
+        context.setUrls(StringUtils.split(urls));
+
+        // raw user search base
+        final PropertyValue rawUserSearchBase = configurationContext.getProperty(PROP_USER_SEARCH_BASE);
+        final PropertyValue rawUserObjectClass = configurationContext.getProperty(PROP_USER_OBJECT_CLASS);
+        final PropertyValue rawUserSearchScope = configurationContext.getProperty(PROP_USER_SEARCH_SCOPE);
+
+        // if loading the users, ensure the object class set
+        if (rawUserSearchBase.isSet() && !rawUserObjectClass.isSet()) {
+            throw new AuthorizerCreationException("LDAP user group provider 'User Object Class' must be specified when 'User Search Base' is set.");
+        }
+
+        // if loading the users, ensure the search scope is set
+        if (rawUserSearchBase.isSet() && !rawUserSearchScope.isSet()) {
+            throw new AuthorizerCreationException("LDAP user group provider 'User Search Scope' must be specified when 'User Search Base' is set.");
+        }
+
+        // user search criteria
+        userSearchBase = rawUserSearchBase.getValue();
+        userObjectClass = rawUserObjectClass.getValue();
+        userSearchFilter = configurationContext.getProperty(PROP_USER_SEARCH_FILTER).getValue();
+        userIdentityAttribute = configurationContext.getProperty(PROP_USER_IDENTITY_ATTRIBUTE).getValue();
+        userGroupNameAttribute = configurationContext.getProperty(PROP_USER_GROUP_ATTRIBUTE).getValue();
+
+        try {
+            userSearchScope = SearchScope.valueOf(rawUserSearchScope.getValue());
+        } catch (final IllegalArgumentException iae) {
+            throw new AuthorizerCreationException(String.format("Unrecognized user search scope '%s'. Possible values are [%s]",
+                    rawUserSearchScope.getValue(), StringUtils.join(SearchScope.values(), ", ")));
+        }
+
+        // determine user behavior
+        useDnForUserIdentity = StringUtils.isBlank(userIdentityAttribute);
+        performUserSearch = StringUtils.isNotBlank(userSearchBase);
+
+        // raw group search criteria
+        final PropertyValue rawGroupSearchBase = configurationContext.getProperty(PROP_GROUP_SEARCH_BASE);
+        final PropertyValue rawGroupObjectClass = configurationContext.getProperty(PROP_GROUP_OBJECT_CLASS);
+        final PropertyValue rawGroupSearchScope = configurationContext.getProperty(PROP_GROUP_SEARCH_SCOPE);
+
+        // if loading the groups, ensure the object class is set
+        if (rawGroupSearchBase.isSet() && !rawGroupObjectClass.isSet()) {
+            throw new AuthorizerCreationException("LDAP user group provider 'Group Object Class' must be specified when 'Group Search Base' is set.");
+        }
+
+        // if loading the groups, ensure the search scope is set
+        if (rawGroupSearchBase.isSet() && !rawGroupSearchScope.isSet()) {
+            throw new AuthorizerCreationException("LDAP user group provider 'Group Search Scope' must be specified when 'Group Search Base' is set.");
+        }
+
+        // group search criteria
+        groupSearchBase = rawGroupSearchBase.getValue();
+        groupObjectClass = rawGroupObjectClass.getValue();
+        groupSearchFilter = configurationContext.getProperty(PROP_GROUP_SEARCH_FILTER).getValue();
+        groupNameAttribute = configurationContext.getProperty(PROP_GROUP_NAME_ATTRIBUTE).getValue();
+        groupMemberAttribute = configurationContext.getProperty(PROP_GROUP_MEMBER_ATTRIBUTE).getValue();
+
+        try {
+            groupSearchScope = SearchScope.valueOf(rawGroupSearchScope.getValue());
+        } catch (final IllegalArgumentException iae) {
+            throw new AuthorizerCreationException(String.format("Unrecognized group search scope '%s'. Possible values are [%s]",
+                    rawGroupSearchScope.getValue(), StringUtils.join(SearchScope.values(), ", ")));
+        }
+
+        // determine group behavior
+        useDnForGroupName = StringUtils.isBlank(groupNameAttribute);
+        performGroupSearch = StringUtils.isNotBlank(groupSearchBase);
+
+        // ensure we are either searching users or groups (at least one must be specified)
+        if (!performUserSearch && !performGroupSearch) {
+            throw new AuthorizerCreationException("LDAP user group provider 'User Search Base' or 'Group Search Base' must be specified.");
+        }
+
+        // ensure group member attribute is set if searching groups but not users
+        if (performGroupSearch && !performUserSearch && StringUtils.isBlank(groupMemberAttribute)) {
+            throw new AuthorizerCreationException("'Group Member Attribute' is required when searching groups but not users.");
+        }
+
+        // get the page size if configured
+        final PropertyValue rawPageSize = configurationContext.getProperty(PROP_PAGE_SIZE);
+        if (rawPageSize.isSet() && StringUtils.isNotBlank(rawPageSize.getValue())) {
+            pageSize = rawPageSize.asInteger();
+        }
+
+        // extract the identity mappings from nifi.properties if any are provided
+        identityMappings = Collections.unmodifiableList(IdentityMappingUtil.getIdentityMappings(properties));
+
+        // set the base environment is necessary
+        if (!baseEnvironment.isEmpty()) {
+            context.setBaseEnvironmentProperties(baseEnvironment);
+        }
+
+        try {
+            // handling initializing beans
+            context.afterPropertiesSet();
+        } catch (final Exception e) {
+            throw new AuthorizerCreationException(e.getMessage(), e);
+        }
+
+        final PropertyValue rawSyncInterval = configurationContext.getProperty(PROP_SYNC_INTERVAL);
+        final long syncInterval;
+        if (rawSyncInterval.isSet()) {
+            try {
+                syncInterval = FormatUtils.getTimeDuration(rawSyncInterval.getValue(), TimeUnit.MILLISECONDS);
+            } catch (final IllegalArgumentException iae) {
+                throw new AuthorizerCreationException(String.format("The %s '%s' is not a valid time duration", PROP_SYNC_INTERVAL, rawSyncInterval.getValue()));
+            }
+        } else {
+            throw new AuthorizerCreationException("The 'Sync Interval' must be specified.");
+        }
+
+        try {
+            // perform the initial load, tenants must be loaded as the configured UserGroupProvider is supplied
+            // to the AccessPolicyProvider for granting initial permissions
+            load(context);
+
+            // ensure the tenants were successfully synced
+            if (tenants.get() == null) {
+                throw new AuthorizerCreationException("Unable to sync users and groups.");
+            }
+
+            // schedule the background thread to load the users/groups
+            ldapSync.scheduleWithFixedDelay(() -> load(context), syncInterval, syncInterval, TimeUnit.SECONDS);
+        } catch (final AuthorizationAccessException e) {
+            throw new AuthorizerCreationException(e);
+        }
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return tenants.get().getAllUsers();
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        return tenants.get().getUsersById().get(identifier);
+    }
+
+    @Override
+    public User getUserByIdentity(String identity) throws AuthorizationAccessException {
+        return tenants.get().getUser(identity);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return tenants.get().getAllGroups();
+    }
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return tenants.get().getGroupsById().get(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(String identity) throws AuthorizationAccessException {
+        final TenantHolder holder = tenants.get();
+        return new UserAndGroups() {
+            @Override
+            public User getUser() {
+                return holder.getUser(identity);
+            }
+
+            @Override
+            public Set<Group> getGroups() {
+                return holder.getGroups(identity);
+            }
+        };
+    }
+
+    /**
+     * Reloads the tenants.
+     */
+    private void load(final ContextSource contextSource) {
+        // create the ldapTemplate based on the context source. use a single source context to use the same connection
+        // to support paging when configured
+        final SingleContextSource singleContextSource = new SingleContextSource(contextSource.getReadOnlyContext());
+        final LdapTemplate ldapTemplate = new LdapTemplate(singleContextSource);
+
+        try {
+            final List<User> userList = new ArrayList<>();
+            final List<Group> groupList = new ArrayList<>();
+
+            // group dn -> user identifiers lookup
+            final Map<String, Set<String>> groupDnToUserIdentifierMappings = new HashMap<>();
+
+            // user dn -> user lookup
+            final Map<String, User> userDnLookup = new HashMap<>();
+
+            if (performUserSearch) {
+                // search controls
+                final SearchControls userControls = new SearchControls();
+                userControls.setSearchScope(userSearchScope.ordinal());
+
+                // consider paging support for users
+                final DirContextProcessor userProcessor;
+                if (pageSize == null) {
+                    userProcessor = new NullDirContextProcessor();
+                } else {
+                    userProcessor = new PagedResultsDirContextProcessor(pageSize);
+                }
+
+                // looking for objects matching the user object class
+                final AndFilter userFilter = new AndFilter();
+                userFilter.and(new EqualsFilter("objectClass", userObjectClass));
+
+                // if a filter has been provided by the user, we add it to the filter
+                if (StringUtils.isNotBlank(userSearchFilter)) {
+                    userFilter.and(new HardcodedFilter(userSearchFilter));
+                }
+
+                do {
+                    userList.addAll(ldapTemplate.search(userSearchBase, userFilter.encode(), userControls, new AbstractContextMapper<User>() {
+                        @Override
+                        protected User doMapFromContext(DirContextOperations ctx) {
+                            final String dn = ctx.getDn().toString();
+
+                            // get the user identity
+                            final String identity = getUserIdentity(ctx);
+
+                            // build the user
+                            final User user = new User.Builder().identifierGenerateFromSeed(identity).identity(identity).build();
+                            userDnLookup.put(dn, user);
+
+                            if (StringUtils.isNotBlank(userGroupNameAttribute)) {
+                                final Attribute attributeGroups = ctx.getAttributes().get(userGroupNameAttribute);
+
+                                if (attributeGroups == null) {
+                                    logger.warn("User group name attribute [" + userGroupNameAttribute + "] does not exist. Ignoring group membership.");
+                                } else {
+                                    try {
+                                        final NamingEnumeration<String> groupDns = (NamingEnumeration<String>) attributeGroups.getAll();
+                                        while (groupDns.hasMoreElements()) {
+                                            // store the group dn -> user identifier mapping
+                                            groupDnToUserIdentifierMappings.computeIfAbsent(groupDns.next(), g -> new HashSet<>()).add(user.getIdentifier());
+                                        }
+                                    } catch (NamingException e) {
+                                        throw new AuthorizationAccessException("Error while retrieving user group name attribute [" + userIdentityAttribute + "].");
+                                    }
+                                }
+                            }
+
+                            return user;
+                        }
+                    }, userProcessor));
+                } while (hasMorePages(userProcessor));
+            }
+
+            if (performGroupSearch) {
+                final SearchControls groupControls = new SearchControls();
+                groupControls.setSearchScope(groupSearchScope.ordinal());
+
+                // consider paging support for groups
+                final DirContextProcessor groupProcessor;
+                if (pageSize == null) {
+                    groupProcessor = new NullDirContextProcessor();
+                } else {
+                    groupProcessor = new PagedResultsDirContextProcessor(pageSize);
+                }
+
+                // looking for objects matching the group object class
+                AndFilter groupFilter = new AndFilter();
+                groupFilter.and(new EqualsFilter("objectClass", groupObjectClass));
+
+                // if a filter has been provided by the user, we add it to the filter
+                if(StringUtils.isNotBlank(groupSearchFilter)) {
+                    groupFilter.and(new HardcodedFilter(groupSearchFilter));
+                }
+
+                do {
+                    groupList.addAll(ldapTemplate.search(groupSearchBase, groupFilter.encode(), groupControls, new AbstractContextMapper<Group>() {
+                        @Override
+                        protected Group doMapFromContext(DirContextOperations ctx) {
+                            final String dn = ctx.getDn().toString();
+
+                            // get the group identity
+                            final String name = getGroupName(ctx);
+
+                            if (!StringUtils.isBlank(groupMemberAttribute)) {
+                                Attribute attributeUsers = ctx.getAttributes().get(groupMemberAttribute);
+                                if (attributeUsers == null) {
+                                    logger.warn("Group member attribute [" + groupMemberAttribute + "] does not exist. Ignoring group membership.");
+                                } else {
+                                    try {
+                                        final NamingEnumeration<String> userDns = (NamingEnumeration<String>) attributeUsers.getAll();
+                                        while (userDns.hasMoreElements()) {
+                                            final String userDn = userDns.next();
+
+                                            if (performUserSearch) {
+                                                // find the user by dn add the identifier to this group
+                                                final User user = userDnLookup.get(userDn);
+
+                                                // ensure the user is known
+                                                if (user != null) {
+                                                    groupDnToUserIdentifierMappings.computeIfAbsent(dn, g -> new HashSet<>()).add(user.getIdentifier());
+                                                } else {
+                                                    logger.warn(String.format("%s contains member %s but that user was not found while searching users. Ignoring group membership.", name, userDn));
+                                                }
+                                            } else {
+                                                final String userIdentity;
+                                                if (useDnForUserIdentity) {
+                                                    // use the dn to avoid the unnecessary look up
+                                                    userIdentity = userDn;
+                                                } else {
+                                                    // lookup the user to extract the user identity
+                                                    userIdentity = getUserIdentity((DirContextAdapter) ldapTemplate.lookup(userDn));
+                                                }
+
+                                                // build the user
+                                                final User user = new User.Builder().identifierGenerateFromSeed(userIdentity).identity(userIdentity).build();
+
+                                                // add this user
+                                                userList.add(user);
+                                                groupDnToUserIdentifierMappings.computeIfAbsent(dn, g -> new HashSet<>()).add(user.getIdentifier());
+                                            }
+                                        }
+                                    } catch (NamingException e) {
+                                        throw new AuthorizationAccessException("Error while retrieving group name attribute [" + groupNameAttribute + "].");
+                                    }
+                                }
+                            }
+
+                            // build this group
+                            final Group.Builder groupBuilder = new Group.Builder().identifierGenerateFromSeed(name).name(name);
+
+                            // add all users that were associated with this group dn
+                            if (groupDnToUserIdentifierMappings.containsKey(dn)) {
+                                groupDnToUserIdentifierMappings.remove(dn).forEach(userIdentifier -> groupBuilder.addUser(userIdentifier));
+                            }
+
+                            return groupBuilder.build();
+                        }
+                    }, groupProcessor));
+                } while (hasMorePages(groupProcessor));
+
+                // any remaining groupDn's were referenced by a user but not found while searching groups
+                groupDnToUserIdentifierMappings.forEach((groupDn, userIdentifiers) -> {
+                    logger.warn(String.format("[%s] are members of %s but that group was not found while searching users. Ignoring group membership.",
+                            StringUtils.join(userIdentifiers, ", "), groupDn));
+                });
+            } else {
+                // groups are not being searched so lookup any groups identified while searching users
+                groupDnToUserIdentifierMappings.forEach((groupDn, userIdentifiers) -> {
+                    final String groupName;
+                    if (useDnForGroupName) {
+                        // use the dn to avoid the unnecessary look up
+                        groupName = groupDn;
+                    } else {
+                        groupName = getGroupName((DirContextAdapter) ldapTemplate.lookup(groupDn));
+                    }
+
+                    // define the group
+                    final Group.Builder groupBuilder = new Group.Builder().identifierGenerateFromSeed(groupName).name(groupName);
+
+                    // add each user
+                    userIdentifiers.forEach(userIdentifier -> groupBuilder.addUser(userIdentifier));
+
+                    // build the group
+                    groupList.add(groupBuilder.build());
+                });
+            }
+
+            // record the updated tenants
+            tenants.set(new TenantHolder(new HashSet<>(userList), new HashSet<>(groupList)));
+        } finally {
+            singleContextSource.destroy();
+        }
+    }
+
+    private boolean hasMorePages(final DirContextProcessor processor ) {
+        return processor instanceof PagedResultsDirContextProcessor && ((PagedResultsDirContextProcessor) processor).hasMore();
+    }
+
+    private String getUserIdentity(final DirContextOperations ctx) {
+        final String identity;
+
+        if (useDnForUserIdentity) {
+            identity = ctx.getDn().toString();
+        } else {
+            final Attribute attributeName = ctx.getAttributes().get(userIdentityAttribute);
+            if (attributeName == null) {
+                throw new AuthorizationAccessException("User identity attribute [" + userIdentityAttribute + "] does not exist.");
+            }
+
+            try {
+                identity = (String) attributeName.get();
+            } catch (NamingException e) {
+                throw new AuthorizationAccessException("Error while retrieving user name attribute [" + userIdentityAttribute + "].");
+            }
+        }
+
+        return IdentityMappingUtil.mapIdentity(identity, identityMappings);
+    }
+
+    private String getGroupName(final DirContextOperations ctx) {
+        final String name;
+
+        if (useDnForGroupName) {
+            name = ctx.getDn().toString();
+        } else {
+            final Attribute attributeName = ctx.getAttributes().get(groupNameAttribute);
+            if (attributeName == null) {
+                throw new AuthorizationAccessException("Group identity attribute [" + groupNameAttribute + "] does not exist.");
+            }
+
+            try {
+                name = (String) attributeName.get();
+            } catch (NamingException e) {
+                throw new AuthorizationAccessException("Error while retrieving group name attribute [" + groupNameAttribute + "].");
+            }
+        }
+
+        return name;
+    }
+
+    @AuthorizerContext
+    public void setNiFiProperties(NiFiRegistryProperties properties) {
+        this.properties = properties;
+    }
+
+    @Override
+    public final void preDestruction() throws ProviderDestructionException {
+        ldapSync.shutdown();
+        try {
+            if (!ldapSync.awaitTermination(10000, TimeUnit.MILLISECONDS)) {
+                logger.info("Failed to stop ldap sync thread in 10 sec. Terminating");
+                ldapSync.shutdownNow();
+            }
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private void setTimeout(final AuthorizerConfigurationContext configurationContext,
+                            final Map<String, Object> baseEnvironment,
+                            final String configurationProperty,
+                            final String environmentKey) {
+
+        final PropertyValue rawTimeout = configurationContext.getProperty(configurationProperty);
+        if (rawTimeout.isSet()) {
+            try {
+                final Long timeout = FormatUtils.getTimeDuration(rawTimeout.getValue(), TimeUnit.MILLISECONDS);
+                baseEnvironment.put(environmentKey, timeout.toString());
+            } catch (final IllegalArgumentException iae) {
+                throw new AuthorizerCreationException(String.format("The %s '%s' is not a valid time duration", configurationProperty, rawTimeout));
+            }
+        }
+    }
+
+    private SSLContext getConfiguredSslContext(final AuthorizerConfigurationContext configurationContext) {
+        final String rawKeystore = configurationContext.getProperty("TLS - Keystore").getValue();
+        final String rawKeystorePassword = configurationContext.getProperty("TLS - Keystore Password").getValue();
+        final String rawKeystoreType = configurationContext.getProperty("TLS - Keystore Type").getValue();
+        final String rawTruststore = configurationContext.getProperty("TLS - Truststore").getValue();
+        final String rawTruststorePassword = configurationContext.getProperty("TLS - Truststore Password").getValue();
+        final String rawTruststoreType = configurationContext.getProperty("TLS - Truststore Type").getValue();
+        final String rawClientAuth = configurationContext.getProperty("TLS - Client Auth").getValue();
+        final String rawProtocol = configurationContext.getProperty("TLS - Protocol").getValue();
+
+        // create the ssl context
+        final SSLContext sslContext;
+        try {
+            if (StringUtils.isBlank(rawKeystore) && StringUtils.isBlank(rawTruststore)) {
+                sslContext = null;
+            } else {
+                // ensure the protocol is specified
+                if (StringUtils.isBlank(rawProtocol)) {
+                    throw new AuthorizerCreationException("TLS - Protocol must be specified.");
+                }
+
+                if (StringUtils.isBlank(rawKeystore)) {
+                    sslContext = SslContextFactory.createTrustSslContext(rawTruststore, rawTruststorePassword.toCharArray(), rawTruststoreType, rawProtocol);
+                } else if (StringUtils.isBlank(rawTruststore)) {
+                    sslContext = SslContextFactory.createSslContext(rawKeystore, rawKeystorePassword.toCharArray(), rawKeystoreType, rawProtocol);
+                } else {
+                    // determine the client auth if specified
+                    final ClientAuth clientAuth;
+                    if (StringUtils.isBlank(rawClientAuth)) {
+                        clientAuth = ClientAuth.NONE;
+                    } else {
+                        try {
+                            clientAuth = ClientAuth.valueOf(rawClientAuth);
+                        } catch (final IllegalArgumentException iae) {
+                            throw new AuthorizerCreationException(String.format("Unrecognized client auth '%s'. Possible values are [%s]",
+                                    rawClientAuth, StringUtils.join(ClientAuth.values(), ", ")));
+                        }
+                    }
+
+                    sslContext = SslContextFactory.createSslContext(rawKeystore, rawKeystorePassword.toCharArray(), rawKeystoreType,
+                            rawTruststore, rawTruststorePassword.toCharArray(), rawTruststoreType, clientAuth, rawProtocol);
+                }
+            }
+        } catch (final KeyStoreException | NoSuchAlgorithmException | CertificateException | UnrecoverableKeyException | KeyManagementException | IOException e) {
+            throw new AuthorizerCreationException(e.getMessage(), e);
+        }
+
+        return sslContext;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/SearchScope.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/SearchScope.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/SearchScope.java
new file mode 100644
index 0000000..2e5e8a2
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/SearchScope.java
@@ -0,0 +1,28 @@
+/*
+ * 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.ldap.tenants;
+
+/**
+ * Scope for searching a directory server.
+ */
+public enum SearchScope {
+
+    OBJECT,
+    ONE_LEVEL,
+    SUBTREE;
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/TenantHolder.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/TenantHolder.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/TenantHolder.java
new file mode 100644
index 0000000..7ef3a8c
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/ldap/tenants/TenantHolder.java
@@ -0,0 +1,165 @@
+/*
+ * 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.ldap.tenants;
+
+
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.User;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A holder to provide atomic access to user group data structures.
+ */
+public class TenantHolder {
+
+    private final Set<User> allUsers;
+    private final Map<String,User> usersById;
+    private final Map<String,User> usersByIdentity;
+
+    private final Set<Group> allGroups;
+    private final Map<String,Group> groupsById;
+    private final Map<String, Set<Group>> groupsByUserIdentity;
+
+    /**
+     * Creates a new holder and populates all convenience data structures.
+     */
+    public TenantHolder(final Set<User> allUsers, final Set<Group> allGroups) {
+        // create a convenience map to retrieve a user by id
+        final Map<String, User> userByIdMap = Collections.unmodifiableMap(createUserByIdMap(allUsers));
+
+        // create a convenience map to retrieve a user by identity
+        final Map<String, User> userByIdentityMap = Collections.unmodifiableMap(createUserByIdentityMap(allUsers));
+
+        // create a convenience map to retrieve a group by id
+        final Map<String, Group> groupByIdMap = Collections.unmodifiableMap(createGroupByIdMap(allGroups));
+
+        // create a convenience map to retrieve the groups for a user identity
+        final Map<String, Set<Group>> groupsByUserIdentityMap = Collections.unmodifiableMap(createGroupsByUserIdentityMap(allGroups, allUsers));
+
+        // set all the holders
+        this.allUsers = allUsers;
+        this.allGroups = allGroups;
+        this.usersById = userByIdMap;
+        this.usersByIdentity = userByIdentityMap;
+        this.groupsById = groupByIdMap;
+        this.groupsByUserIdentity = groupsByUserIdentityMap;
+    }
+
+    /**
+     * Creates a Map from user identifier to User.
+     *
+     * @param users the set of all users
+     * @return the Map from user identifier to User
+     */
+    private Map<String,User> createUserByIdMap(final Set<User> users) {
+        Map<String,User> usersMap = new HashMap<>();
+        for (User user : users) {
+            usersMap.put(user.getIdentifier(), user);
+        }
+        return usersMap;
+    }
+
+    /**
+     * Creates a Map from user identity to User.
+     *
+     * @param users the set of all users
+     * @return the Map from user identity to User
+     */
+    private Map<String,User> createUserByIdentityMap(final Set<User> users) {
+        Map<String,User> usersMap = new HashMap<>();
+        for (User user : users) {
+            usersMap.put(user.getIdentity(), user);
+        }
+        return usersMap;
+    }
+
+    /**
+     * Creates a Map from group identifier to Group.
+     *
+     * @param groups the set of all groups
+     * @return the Map from group identifier to Group
+     */
+    private Map<String,Group> createGroupByIdMap(final Set<Group> groups) {
+        Map<String,Group> groupsMap = new HashMap<>();
+        for (Group group : groups) {
+            groupsMap.put(group.getIdentifier(), group);
+        }
+        return groupsMap;
+    }
+
+    /**
+     * Creates a Map from user identity to the set of Groups for that identity.
+     *
+     * @param groups all groups
+     * @param users all users
+     * @return a Map from User identity to the set of Groups for that identity
+     */
+    private Map<String, Set<Group>> createGroupsByUserIdentityMap(final Set<Group> groups, final Set<User> users) {
+        Map<String, Set<Group>> groupsByUserIdentity = new HashMap<>();
+
+        for (User user : users) {
+            Set<Group> userGroups = new HashSet<>();
+            for (Group group : groups) {
+                for (String groupUser : group.getUsers()) {
+                    if (groupUser.equals(user.getIdentifier())) {
+                        userGroups.add(group);
+                    }
+                }
+            }
+
+            groupsByUserIdentity.put(user.getIdentity(), userGroups);
+        }
+
+        return groupsByUserIdentity;
+    }
+
+    Set<User> getAllUsers() {
+        return allUsers;
+    }
+
+    Map<String, User> getUsersById() {
+        return usersById;
+    }
+
+    Set<Group> getAllGroups() {
+        return allGroups;
+    }
+
+    Map<String, Group> getGroupsById() {
+        return groupsById;
+    }
+
+    public User getUser(String identity) {
+        if (identity == null) {
+            throw new IllegalArgumentException("Identity cannot be null");
+        }
+        return usersByIdentity.get(identity);
+    }
+
+    public Set<Group> getGroups(String userIdentity) {
+        if (userIdentity == null) {
+            throw new IllegalArgumentException("User Identity cannot be null");
+        }
+        return groupsByUserIdentity.get(userIdentity);
+    }
+
+}


[17/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
NIFIREG-33 Add LDAP and JWT auth support

- Adds LdapIdentityProvider for authentication
- Adds /access/token endpoint for generating JWT for users that can authenticate with a configured IdenitiyProvider
- Adds JwtAuthenticationProvider for authentication
- Adds KeyService for key generation and tracking for signing JWTs
- Adds LdapUserGroupProvider for authorization
- Adds LDAP integration tests
- Refactors nifi-registry-security-api-impl into nifi-registry-framework
- Refactors all security related packages, such as o.a.n.r.authorization and o.a.n.r.authentication, under org.apache.nifi.registry.security
- Fixing issues found in code review
- Make LoginIdentityProviderFactory a DisposableBean

This closes #29.

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

Branch: refs/heads/master
Commit: 90f36dd223d50a0d29eef4eed24776e64b82c869
Parents: a43e81f
Author: Kevin Doran <kd...@gmail.com>
Authored: Thu Oct 12 13:54:34 2017 -0400
Committer: Bryan Bende <bb...@apache.org>
Committed: Tue Nov 7 13:49:58 2017 -0500

----------------------------------------------------------------------
 nifi-registry-assembly/pom.xml                  |   5 +-
 .../client/NiFiRegistryClientConfig.java        |   8 +-
 nifi-registry-framework/pom.xml                 |  80 +-
 .../authorization/AuthorizableLookup.java       |  83 --
 .../registry/authorization/AuthorizeAccess.java |  21 -
 .../AuthorizerCapabilityDetection.java          |  75 --
 .../authorization/AuthorizerFactory.java        |  33 -
 .../AuthorizerFactoryException.java             |  33 -
 .../CompositeConfigurableUserGroupProvider.java | 197 -----
 .../CompositeUserGroupProvider.java             | 177 ----
 .../StandardAuthorizableLookup.java             | 237 ------
 .../StandardAuthorizerFactory.java              | 820 ------------------
 .../StandardManagedAuthorizer.java              | 264 ------
 .../resource/AccessPolicyAuthorizable.java      | 122 ---
 .../authorization/resource/Authorizable.java    | 300 -------
 ...rcePolicyPermissionsThroughBaseResource.java |  36 -
 .../authorization/resource/ResourceFactory.java | 261 ------
 .../authorization/resource/ResourceType.java    |  52 --
 .../registry/authorization/user/NiFiUser.java   |  52 --
 .../authorization/user/NiFiUserDetails.java     |  91 --
 .../authorization/user/NiFiUserUtils.java       |  91 --
 .../authorization/user/StandardNiFiUser.java    | 189 -----
 .../nifi/registry/db/DatabaseKeyService.java    | 117 +++
 .../nifi/registry/db/entity/KeyEntity.java      |  61 ++
 .../registry/db/repository/KeyRepository.java   |  31 +
 .../registry/extension/ExtensionManager.java    |   8 +-
 .../AbstractPolicyBasedAuthorizer.java          | 839 +++++++++++++++++++
 .../authorization/AuthorizableLookup.java       |  83 ++
 .../security/authorization/AuthorizeAccess.java |  21 +
 .../AuthorizerCapabilityDetection.java          |  84 ++
 .../authorization/AuthorizerFactory.java        | 809 ++++++++++++++++++
 .../AuthorizerFactoryException.java             |  33 +
 .../CompositeConfigurableUserGroupProvider.java | 205 +++++
 .../CompositeUserGroupProvider.java             | 184 ++++
 .../StandardAuthorizableLookup.java             | 239 ++++++
 .../StandardAuthorizerConfigurationContext.java |  54 ++
 ...StandardAuthorizerInitializationContext.java |  52 ++
 .../StandardManagedAuthorizer.java              | 278 ++++++
 .../authorization/UsersAndAccessPolicies.java   |  52 ++
 .../file/AuthorizationsHolder.java              | 187 +++++
 .../file/FileAccessPolicyProvider.java          | 757 +++++++++++++++++
 .../authorization/file/FileAuthorizer.java      | 288 +++++++
 .../file/FileUserGroupProvider.java             | 775 +++++++++++++++++
 .../authorization/file/IdentifierUtil.java      |  35 +
 .../authorization/file/UserGroupHolder.java     | 241 ++++++
 .../resource/AccessPolicyAuthorizable.java      | 122 +++
 .../authorization/resource/Authorizable.java    | 300 +++++++
 ...rcePolicyPermissionsThroughBaseResource.java |  36 +
 .../authorization/resource/ResourceFactory.java | 261 ++++++
 .../authorization/resource/ResourceType.java    |  52 ++
 .../security/authorization/user/NiFiUser.java   |  52 ++
 .../authorization/user/NiFiUserDetails.java     |  91 ++
 .../authorization/user/NiFiUserUtils.java       |  91 ++
 .../authorization/user/StandardNiFiUser.java    | 189 +++++
 .../apache/nifi/registry/security/key/Key.java  |  69 ++
 .../nifi/registry/security/key/KeyService.java  |  46 +
 .../security/ldap/IdentityStrategy.java         |  22 +
 .../ldap/LdapAuthenticationStrategy.java        |  24 +
 .../security/ldap/LdapIdentityProvider.java     | 348 ++++++++
 .../security/ldap/LdapsSocketFactory.java       | 106 +++
 .../security/ldap/ReferralStrategy.java         |  35 +
 .../ldap/tenants/LdapUserGroupProvider.java     | 750 +++++++++++++++++
 .../security/ldap/tenants/SearchScope.java      |  28 +
 .../security/ldap/tenants/TenantHolder.java     | 165 ++++
 .../nifi/registry/security/util/XmlUtils.java   |  44 +
 .../registry/service/AuthorizationService.java  | 144 ++--
 .../nifi/registry/service/DataModelMapper.java  |  18 +
 ...pache.nifi.registry.authorization.Authorizer |  15 -
 ...ifi.registry.authorization.UserGroupProvider |  16 -
 ...ecurity.authentication.LoginIdentityProvider |  15 +
 ....security.authorization.AccessPolicyProvider |  15 +
 ...i.registry.security.authorization.Authorizer |  16 +
 ...try.security.authorization.UserGroupProvider |  18 +
 .../main/resources/db/migration/V1__Initial.sql |   7 +
 .../src/main/xsd/authorizations.xsd             |  87 ++
 .../src/main/xsd/tenants.xsd                    |  96 +++
 .../registry/db/DatabaseKeyServiceSpec.groovy   |  85 ++
 .../service/AuthorizationServiceSpec.groovy     |  22 +-
 .../db/repository/TestKeyRepository.java        | 106 +++
 .../db/migration/V999999.1__test-setup.sql      |   8 +-
 .../properties/NiFiRegistryProperties.java      |  12 +
 .../src/main/resources/conf/authorizers.xml     | 106 ++-
 .../main/resources/conf/identity-providers.xml  |  90 ++
 .../resources/conf/nifi-registry.properties     |   2 +
 nifi-registry-security-api-impl/pom.xml         | 100 ---
 .../AbstractPolicyBasedAuthorizer.java          | 824 ------------------
 .../StandardAuthorizerConfigurationContext.java |  54 --
 ...StandardAuthorizerInitializationContext.java |  55 --
 .../authorization/UsersAndAccessPolicies.java   |  52 --
 .../annotation/AuthorizerContext.java           |  35 -
 .../file/AuthorizationsHolder.java              | 187 -----
 .../file/FileAccessPolicyProvider.java          | 757 -----------------
 .../authorization/file/FileAuthorizer.java      | 288 -------
 .../file/FileUserGroupProvider.java             | 775 -----------------
 .../authorization/file/IdentifierUtil.java      |  35 -
 .../authorization/file/UserGroupHolder.java     | 241 ------
 ....registry.authorization.AccessPolicyProvider |  15 -
 ...pache.nifi.registry.authorization.Authorizer |  15 -
 ...ifi.registry.authorization.UserGroupProvider |  15 -
 .../src/main/xsd/authorizations.xsd             |  87 --
 .../src/main/xsd/tenants.xsd                    |  96 ---
 .../registry/authorization/AccessPolicy.java    | 367 --------
 .../authorization/AccessPolicyProvider.java     |  90 --
 ...cessPolicyProviderInitializationContext.java |  30 -
 .../AccessPolicyProviderLookup.java             |  31 -
 .../authorization/AuthorizationAuditor.java     |  30 -
 .../authorization/AuthorizationRequest.java     | 245 ------
 .../authorization/AuthorizationResult.java      | 103 ---
 .../nifi/registry/authorization/Authorizer.java |  63 --
 .../AuthorizerConfigurationContext.java         |  48 --
 .../AuthorizerInitializationContext.java        |  30 -
 .../authorization/AuthorizerLookup.java         |  31 -
 .../ConfigurableAccessPolicyProvider.java       | 108 ---
 .../ConfigurableUserGroupProvider.java          | 163 ----
 .../nifi/registry/authorization/Group.java      | 263 ------
 .../authorization/ManagedAuthorizer.java        |  59 --
 .../registry/authorization/RequestAction.java   |  56 --
 .../nifi/registry/authorization/Resource.java   |  44 -
 .../nifi/registry/authorization/User.java       | 188 -----
 .../registry/authorization/UserAndGroups.java   |  40 -
 .../registry/authorization/UserContextKeys.java |  26 -
 .../authorization/UserGroupProvider.java        | 108 ---
 .../UserGroupProviderInitializationContext.java |  37 -
 .../authorization/UserGroupProviderLookup.java  |  31 -
 .../exception/AccessDeniedException.java        |  39 -
 .../exception/AuthorizationAccessException.java |  32 -
 .../exception/AuthorizerCreationException.java  |  39 -
 .../AuthorizerDestructionException.java         |  39 -
 .../UninheritableAuthorizationsException.java   |  28 -
 .../authentication/AuthenticationResponse.java  |  65 ++
 .../authentication/LoginCredentials.java        |  39 +
 .../authentication/LoginIdentityProvider.java   |  61 ++
 ...ginIdentityProviderConfigurationContext.java |  48 ++
 ...inIdentityProviderInitializationContext.java |  27 +
 .../LoginIdentityProviderLookup.java            |  23 +
 .../LoginIdentityProviderContext.java           |  35 +
 .../exception/IdentityAccessException.java      |  33 +
 .../InvalidLoginCredentialsException.java       |  33 +
 .../exception/ProviderCreationException.java    |  39 +
 .../exception/ProviderDestructionException.java |  39 +
 .../security/authorization/AccessPolicy.java    | 367 ++++++++
 .../authorization/AccessPolicyProvider.java     |  90 ++
 ...cessPolicyProviderInitializationContext.java |  30 +
 .../AccessPolicyProviderLookup.java             |  31 +
 .../authorization/AuthorizationAuditor.java     |  30 +
 .../authorization/AuthorizationRequest.java     | 245 ++++++
 .../authorization/AuthorizationResult.java      | 103 +++
 .../security/authorization/Authorizer.java      |  63 ++
 .../AuthorizerConfigurationContext.java         |  48 ++
 .../AuthorizerInitializationContext.java        |  30 +
 .../authorization/AuthorizerLookup.java         |  31 +
 .../ConfigurableAccessPolicyProvider.java       | 108 +++
 .../ConfigurableUserGroupProvider.java          | 163 ++++
 .../registry/security/authorization/Group.java  | 263 ++++++
 .../authorization/ManagedAuthorizer.java        |  59 ++
 .../security/authorization/RequestAction.java   |  56 ++
 .../security/authorization/Resource.java        |  44 +
 .../registry/security/authorization/User.java   | 188 +++++
 .../security/authorization/UserAndGroups.java   |  40 +
 .../security/authorization/UserContextKeys.java |  26 +
 .../authorization/UserGroupProvider.java        | 108 +++
 .../UserGroupProviderInitializationContext.java |  37 +
 .../authorization/UserGroupProviderLookup.java  |  31 +
 .../annotation/AuthorizerContext.java           |  35 +
 .../exception/AccessDeniedException.java        |  39 +
 .../exception/AuthorizationAccessException.java |  32 +
 .../exception/AuthorizerCreationException.java  |  39 +
 .../AuthorizerDestructionException.java         |  39 +
 .../UninheritableAuthorizationsException.java   |  28 +
 .../security/util/CertificateUtils.java         |   2 +
 .../security/util/SslContextFactory.java        | 249 ++++++
 nifi-registry-web-api/pom.xml                   |  38 +-
 .../registry/NiFiRegistryApiApplication.java    |   8 +-
 .../web/NiFiRegistryResourceConfig.java         |  13 +
 .../web/NiFiRegistrySecurityConfig.java         | 221 -----
 .../registry/web/api/AccessPolicyResource.java  |  25 +-
 .../nifi/registry/web/api/AccessResource.java   | 203 +++--
 .../api/AuthorizableApplicationResource.java    |  10 +-
 .../registry/web/api/BucketFlowResource.java    |   4 +-
 .../nifi/registry/web/api/BucketResource.java   |   8 +-
 .../nifi/registry/web/api/FlowResource.java     |   2 +-
 .../nifi/registry/web/api/ItemResource.java     |   4 +-
 .../nifi/registry/web/api/ResourceResource.java |   8 +-
 .../nifi/registry/web/api/TenantResource.java   |  10 +-
 .../web/mapper/AccessDeniedExceptionMapper.java |   6 +-
 .../AuthorizationAccessExceptionMapper.java     |   2 +-
 .../InvalidAuthenticationExceptionMapper.java   |   2 +-
 .../InvalidAuthenticationException.java         |  35 -
 .../web/security/NiFiAnonymousUserFilter.java   |  40 -
 .../web/security/NiFiAuthenticationFilter.java  | 154 ----
 .../security/NiFiAuthenticationProvider.java    |  84 --
 .../NiFiAuthenticationRequestToken.java         |  41 -
 .../security/NiFiRegistrySecurityConfig.java    | 158 ++++
 .../web/security/ProxiedEntitiesUtils.java      | 163 ----
 .../web/security/UntrustedProxyException.java   |  34 -
 .../LoginIdentityProviderFactory.java           | 264 ++++++
 .../authentication/NiFiAnonymousUserFilter.java |  40 +
 .../NiFiAuthenticationFilter.java               | 156 ++++
 .../NiFiAuthenticationProvider.java             |  84 ++
 .../NiFiAuthenticationRequestToken.java         |  41 +
 .../authentication/ProxiedEntitiesUtils.java    | 163 ++++
 ...ginIdentityProviderConfigurationContext.java |  52 ++
 ...inIdentityProviderInitializationContext.java |  45 +
 .../InvalidAuthenticationException.java         |  35 +
 .../exception/UntrustedProxyException.java      |  34 +
 .../jwt/JwtAuthenticationFilter.java            |  58 ++
 .../jwt/JwtAuthenticationProvider.java          |  69 ++
 .../jwt/JwtAuthenticationRequestToken.java      |  60 ++
 .../security/authentication/jwt/JwtService.java | 160 ++++
 .../token/LoginAuthenticationToken.java         | 123 +++
 .../token/NiFiAuthenticationToken.java          |  55 ++
 .../x509/SubjectDnX509PrincipalExtractor.java   |  35 +
 .../x509/X509AuthenticationFilter.java          |  64 ++
 .../x509/X509AuthenticationProvider.java        | 166 ++++
 .../x509/X509AuthenticationRequestToken.java    |  75 ++
 .../x509/X509CertificateExtractor.java          |  55 ++
 .../x509/X509CertificateValidator.java          |  49 ++
 .../x509/X509IdentityProvider.java              | 104 +++
 .../token/LoginAuthenticationToken.java         | 123 ---
 .../security/token/NiFiAuthenticationToken.java |  55 --
 .../security/token/OtpAuthenticationToken.java  |  56 --
 .../x509/SubjectDnX509PrincipalExtractor.java   |  33 -
 .../security/x509/X509AuthenticationFilter.java |  64 --
 .../x509/X509AuthenticationProvider.java        | 165 ----
 .../x509/X509AuthenticationRequestToken.java    |  75 --
 .../security/x509/X509CertificateExtractor.java |  53 --
 .../security/x509/X509CertificateValidator.java |  47 --
 .../web/security/x509/X509IdentityProvider.java |  95 ---
 .../src/main/xsd/identity-providers.xsd         |  50 ++
 .../NiFiRegistryApiTestApplication.java         |  59 --
 .../NiFiRegistryTestApiApplication.java         |  40 +
 .../registry/SecureLdapTestApiApplication.java  |  45 +
 .../nifi/registry/web/api/SecureFileIT.java     |   4 +-
 .../nifi/registry/web/api/SecureLdapIT.java     | 230 +++++
 .../nifi/registry/web/api/UnsecuredITBase.java  |   4 +-
 .../application-ITSecureLdap.properties         |  48 ++
 .../src/test/resources/application.properties   |   8 +-
 .../resources/conf/secure-file/authorizers.xml  |   6 +-
 .../conf/secure-file/nifi-registry.properties   |   3 -
 .../resources/conf/secure-ldap/authorizers.xml  | 243 ++++++
 .../conf/secure-ldap/identity-providers.xml     |  88 ++
 .../secure-ldap/nifi-registry-client.properties |  25 +
 .../conf/secure-ldap/nifi-registry.properties   |  32 +
 .../conf/secure-ldap/test-ldap-data.ldif        | 261 ++++++
 pom.xml                                         |   2 +-
 245 files changed, 15669 insertions(+), 10969 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-assembly/pom.xml b/nifi-registry-assembly/pom.xml
index 69a5ec0..41f66b8 100644
--- a/nifi-registry-assembly/pom.xml
+++ b/nifi-registry-assembly/pom.xml
@@ -151,8 +151,11 @@
         <nifi.registry.security.truststore />
         <nifi.registry.security.truststoreType />
         <nifi.registry.security.truststorePasswd />
-        <nifi.registry.security.authorizers.configuration.file>./conf/authorizers.xml</nifi.registry.security.authorizers.configuration.file><nifi.registry.security.needClientAuth />
+        <nifi.registry.security.needClientAuth />
+        <nifi.registry.security.authorizers.configuration.file>./conf/authorizers.xml</nifi.registry.security.authorizers.configuration.file>
         <nifi.registry.security.authorizer>managed-authorizer</nifi.registry.security.authorizer>
+        <nifi.registry.security.identity.providers.configuration.file>./conf/identity-providers.xml</nifi.registry.security.identity.providers.configuration.file>
+        <nifi.registry.security.identity.provider />
 
         <!-- nifi-registry.properties: provider properties -->
         <nifi.registry.providers.configuration.file>./conf/providers.xml</nifi.registry.providers.configuration.file>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClientConfig.java
----------------------------------------------------------------------
diff --git a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClientConfig.java b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClientConfig.java
index 6d5ddb1..de77b51 100644
--- a/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClientConfig.java
+++ b/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClientConfig.java
@@ -20,8 +20,10 @@ import org.apache.nifi.registry.security.util.KeyStoreUtils;
 import org.apache.nifi.registry.security.util.KeystoreType;
 
 import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.KeyManager;
 import javax.net.ssl.KeyManagerFactory;
 import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManager;
 import javax.net.ssl.TrustManagerFactory;
 import java.io.File;
 import java.io.FileInputStream;
@@ -111,11 +113,13 @@ public class NiFiRegistryClientConfig {
             trustManagerFactory = null;
         }
 
-        if (keyManagerFactory != null && trustManagerFactory != null) {
+        if (keyManagerFactory != null || trustManagerFactory != null) {
             try {
                 // initialize the ssl context
+                KeyManager[] keyManagers = keyManagerFactory != null ? keyManagerFactory.getKeyManagers() : null;
+                TrustManager[] trustManagers = trustManagerFactory != null ? trustManagerFactory.getTrustManagers() : null;
                 final SSLContext sslContext = SSLContext.getInstance("TLS");
-                sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), new SecureRandom());
+                sslContext.init(keyManagers, trustManagers, new SecureRandom());
                 sslContext.getDefaultSSLParameters().setNeedClientAuth(true);
 
                 return sslContext;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/pom.xml b/nifi-registry-framework/pom.xml
index 1179188..95381f5 100644
--- a/nifi-registry-framework/pom.xml
+++ b/nifi-registry-framework/pom.xml
@@ -44,7 +44,11 @@
                             <goal>xjc</goal>
                         </goals>
                         <configuration>
+                            <sources>
+                                <source>src/main/xsd/providers.xsd</source>
+                            </sources>
                             <packageName>org.apache.nifi.registry.provider.generated</packageName>
+                            <clearOutputDir>false</clearOutputDir>
                         </configuration>
                     </execution>
                     <execution>
@@ -53,7 +57,36 @@
                             <goal>xjc</goal>
                         </goals>
                         <configuration>
-                            <packageName>org.apache.nifi.registry.authorization.generated</packageName>
+                            <sources>
+                                <source>src/main/xsd/authorizers.xsd</source>
+                            </sources>
+                            <packageName>org.apache.nifi.registry.security.authorization.generated</packageName>
+                            <clearOutputDir>false</clearOutputDir>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>authorizations</id>
+                        <goals>
+                            <goal>xjc</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/xsd/authorizations.xsd</source>
+                            </sources>
+                            <packageName>org.apache.nifi.registry.security.authorization.file.generated</packageName>
+                            <clearOutputDir>false</clearOutputDir>
+                        </configuration>
+                    </execution>
+                    <execution>
+                        <id>tenants</id>
+                        <goals>
+                            <goal>xjc</goal>
+                        </goals>
+                        <configuration>
+                            <sources>
+                                <source>src/main/xsd/tenants.xsd</source>
+                            </sources>
+                            <packageName>org.apache.nifi.registry.security.authorization.file.tenants.generated</packageName>
                             <clearOutputDir>false</clearOutputDir>
                         </configuration>
                     </execution>
@@ -63,7 +96,7 @@
                 <groupId>org.apache.maven.plugins</groupId>
                 <artifactId>maven-checkstyle-plugin</artifactId>
                 <configuration>
-                    <excludes>**/authorization/generated/*.java,**/provider/generated/*.java,</excludes>
+                    <excludes>**/generated/*.java</excludes>
                 </configuration>
             </plugin>
             <plugin>
@@ -120,7 +153,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-security-api-impl</artifactId>
+            <artifactId>nifi-registry-security-utils</artifactId>
             <version>0.0.1-SNAPSHOT</version>
         </dependency>
         <dependency>
@@ -129,6 +162,42 @@
             <version>${spring.boot.version}</version>
         </dependency>
         <dependency>
+            <groupId>org.springframework.security</groupId>
+            <artifactId>spring-security-ldap</artifactId>
+            <version>${spring.security.version}</version>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.springframework.security</groupId>
+                    <artifactId>spring-security-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.springframework</groupId>
+                    <artifactId>spring-beans</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.springframework</groupId>
+                    <artifactId>spring-context</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.springframework</groupId>
+                    <artifactId>spring-core</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>org.springframework</groupId>
+                    <artifactId>spring-tx</artifactId>
+                </exclusion>
+                <exclusion>
+                    <groupId>commons-logging</groupId>
+                    <artifactId>commons-logging</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
+        <dependency>
+            <groupId>org.bouncycastle</groupId>
+            <artifactId>bcprov-jdk15on</artifactId>
+            <version>1.55</version>
+        </dependency>
+        <dependency>
             <groupId>commons-io</groupId>
             <artifactId>commons-io</artifactId>
         </dependency>
@@ -199,10 +268,5 @@
             <version>2.2.2</version>
             <scope>test</scope>
         </dependency>
-        <dependency>
-            <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-security-utils</artifactId>
-            <version>0.0.1-SNAPSHOT</version>
-        </dependency>
     </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizableLookup.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizableLookup.java
deleted file mode 100644
index e5016d3..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizableLookup.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.nifi.registry.authorization.resource.Authorizable;
-
-public interface AuthorizableLookup {
-
-    /**
-     * Get the authorizable for retrieving resources.
-     *
-     * @return authorizable
-     */
-    Authorizable getResourcesAuthorizable();
-
-    /**
-     * Get the authorizable for /proxy.
-     *
-     * @return authorizable
-     */
-    Authorizable getProxyAuthorizable();
-
-    /**
-     * Get the authorizable for all tenants.
-     *
-     * Get the {@link Authorizable} that represents the resource of users and user groups.
-     * @return authorizable
-     */
-    Authorizable getTenantsAuthorizable();
-
-    /**
-     * Get the authorizable for all access policies.
-     *
-     * @return authorizable
-     */
-    Authorizable getPoliciesAuthorizable();
-
-    /**
-     * Get the authorizable for all Buckets.
-     *
-     * @return authorizable
-     */
-    Authorizable getBucketsAuthorizable();
-
-    /**
-     * Get the authorizable for the Bucket with the bucket id.
-     *
-     * @param bucketIdentifier bucket id
-     * @return authorizable
-     */
-    Authorizable getBucketAuthorizable(String bucketIdentifier);
-
-    /**
-     * Get the authorizable for the policy of the specified resource.
-     *
-     * @param resource resource
-     * @return authorizable
-     */
-    Authorizable getAccessPolicyByResource(String resource);
-
-    /**
-     * Get the authorizable of the specified resource.
-     *
-     * @param resource resource
-     * @return authorizable
-     */
-    Authorizable getAuthorizableByResource(final String resource);
-
-}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizerCapabilityDetection.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizerCapabilityDetection.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizerCapabilityDetection.java
deleted file mode 100644
index e6cf79f..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizerCapabilityDetection.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-public final class AuthorizerCapabilityDetection {
-
-    public static boolean isManagedAuthorizer(final Authorizer authorizer) {
-        return authorizer instanceof ManagedAuthorizer;
-    }
-
-    public static boolean isConfigurableAccessPolicyProvider(final Authorizer authorizer) {
-        if (!isManagedAuthorizer(authorizer)) {
-            return false;
-        }
-
-        final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
-        return managedAuthorizer.getAccessPolicyProvider() instanceof ConfigurableAccessPolicyProvider;
-    }
-
-    public static boolean isConfigurableUserGroupProvider(final Authorizer authorizer) {
-        if (!isManagedAuthorizer(authorizer)) {
-            return false;
-        }
-
-        final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
-        final AccessPolicyProvider accessPolicyProvider = managedAuthorizer.getAccessPolicyProvider();
-        return accessPolicyProvider.getUserGroupProvider() instanceof ConfigurableUserGroupProvider;
-    }
-
-    public static boolean isUserConfigurable(final Authorizer authorizer, final User user) {
-        if (!isConfigurableUserGroupProvider(authorizer)) {
-            return false;
-        }
-
-        final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
-        final ConfigurableUserGroupProvider configurableUserGroupProvider = (ConfigurableUserGroupProvider) managedAuthorizer.getAccessPolicyProvider().getUserGroupProvider();
-        return configurableUserGroupProvider.isConfigurable(user);
-    }
-
-    public static boolean isGroupConfigurable(final Authorizer authorizer, final Group group) {
-        if (!isConfigurableUserGroupProvider(authorizer)) {
-            return false;
-        }
-
-        final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
-        final ConfigurableUserGroupProvider configurableUserGroupProvider = (ConfigurableUserGroupProvider) managedAuthorizer.getAccessPolicyProvider().getUserGroupProvider();
-        return configurableUserGroupProvider.isConfigurable(group);
-    }
-
-    public static boolean isAccessPolicyConfigurable(final Authorizer authorizer, final AccessPolicy accessPolicy) {
-        if (!isConfigurableAccessPolicyProvider(authorizer)) {
-            return false;
-        }
-
-        final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
-        final ConfigurableAccessPolicyProvider configurableAccessPolicyProvider = (ConfigurableAccessPolicyProvider) managedAuthorizer.getAccessPolicyProvider();
-        return configurableAccessPolicyProvider.isConfigurable(accessPolicy);
-    }
-
-    private AuthorizerCapabilityDetection() {}
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizerFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizerFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizerFactory.java
deleted file mode 100644
index 9cace73..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/AuthorizerFactory.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-public interface AuthorizerFactory {
-
-    /**
-     * Initialize the factory.
-     *
-     * @throws AuthorizerFactoryException if an error occurs during initialization
-     */
-    void initialize() throws AuthorizerFactoryException;
-
-    /**
-     * @return the configured Authorizer
-     */
-    Authorizer getAuthorizer() throws AuthorizerFactoryException;
-
-}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/CompositeConfigurableUserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/CompositeConfigurableUserGroupProvider.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/CompositeConfigurableUserGroupProvider.java
deleted file mode 100644
index 9664558..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/CompositeConfigurableUserGroupProvider.java
+++ /dev/null
@@ -1,197 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.nifi.registry.authorization.exception.UninheritableAuthorizationsException;
-import org.apache.nifi.registry.util.PropertyValue;
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerDestructionException;
-
-import java.util.HashSet;
-import java.util.Set;
-
-public class CompositeConfigurableUserGroupProvider extends CompositeUserGroupProvider implements ConfigurableUserGroupProvider {
-
-    static final String PROP_CONFIGURABLE_USER_GROUP_PROVIDER = "Configurable User Group Provider";
-
-    private UserGroupProviderLookup userGroupProviderLookup;
-    private ConfigurableUserGroupProvider configurableUserGroupProvider;
-
-    public CompositeConfigurableUserGroupProvider() {
-        super(true);
-    }
-
-    @Override
-    public void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
-        userGroupProviderLookup = initializationContext.getUserGroupProviderLookup();
-
-        // initialize the CompositeUserGroupProvider
-        super.initialize(initializationContext);
-    }
-
-    @Override
-    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-        final PropertyValue configurableUserGroupProviderKey = configurationContext.getProperty(PROP_CONFIGURABLE_USER_GROUP_PROVIDER);
-        if (!configurableUserGroupProviderKey.isSet()) {
-            throw new AuthorizerCreationException("The Configurable User Group Provider must be set.");
-        }
-
-        final UserGroupProvider userGroupProvider = userGroupProviderLookup.getUserGroupProvider(configurableUserGroupProviderKey.getValue());
-
-        if (userGroupProvider == null) {
-            throw new AuthorizerCreationException(String.format("Unable to locate the Configurable User Group Provider: %s", configurableUserGroupProviderKey));
-        }
-
-        if (!(userGroupProvider instanceof ConfigurableUserGroupProvider)) {
-            throw new AuthorizerCreationException(String.format("The Configurable User Group Provider is not configurable: %s", configurableUserGroupProviderKey));
-        }
-
-        configurableUserGroupProvider = (ConfigurableUserGroupProvider) userGroupProvider;
-
-        // configure the CompositeUserGroupProvider
-        super.onConfigured(configurationContext);
-    }
-
-    @Override
-    public String getFingerprint() throws AuthorizationAccessException {
-        return configurableUserGroupProvider.getFingerprint();
-    }
-
-    @Override
-    public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
-        configurableUserGroupProvider.inheritFingerprint(fingerprint);
-    }
-
-    @Override
-    public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
-        configurableUserGroupProvider.checkInheritability(proposedFingerprint);
-    }
-
-    @Override
-    public User addUser(User user) throws AuthorizationAccessException {
-        return configurableUserGroupProvider.addUser(user);
-    }
-
-    @Override
-    public boolean isConfigurable(User user) {
-        return configurableUserGroupProvider.isConfigurable(user);
-    }
-
-    @Override
-    public User updateUser(User user) throws AuthorizationAccessException {
-        return configurableUserGroupProvider.updateUser(user);
-    }
-
-    @Override
-    public User deleteUser(User user) throws AuthorizationAccessException {
-        return configurableUserGroupProvider.deleteUser(user);
-    }
-
-    @Override
-    public User deleteUser(String userIdentifier) throws AuthorizationAccessException {
-        return configurableUserGroupProvider.deleteUser(userIdentifier);
-    }
-
-    @Override
-    public Group addGroup(Group group) throws AuthorizationAccessException {
-        return configurableUserGroupProvider.addGroup(group);
-    }
-
-    @Override
-    public boolean isConfigurable(Group group) {
-        return configurableUserGroupProvider.isConfigurable(group);
-    }
-
-    @Override
-    public Group updateGroup(Group group) throws AuthorizationAccessException {
-        return configurableUserGroupProvider.updateGroup(group);
-    }
-
-    @Override
-    public Group deleteGroup(Group group) throws AuthorizationAccessException {
-        return configurableUserGroupProvider.deleteGroup(group);
-    }
-
-    @Override
-    public Group deleteGroup(String groupIdentifier) throws AuthorizationAccessException {
-        return configurableUserGroupProvider.deleteGroup(groupIdentifier);
-    }
-
-    @Override
-    public Set<User> getUsers() throws AuthorizationAccessException {
-        final Set<User> users = new HashSet<>(configurableUserGroupProvider.getUsers());
-        users.addAll(super.getUsers());
-        return users;
-    }
-
-    @Override
-    public User getUser(String identifier) throws AuthorizationAccessException {
-        User user = configurableUserGroupProvider.getUser(identifier);
-
-        if (user == null) {
-            user = super.getUser(identifier);
-        }
-
-        return user;
-    }
-
-    @Override
-    public User getUserByIdentity(String identity) throws AuthorizationAccessException {
-        User user = configurableUserGroupProvider.getUserByIdentity(identity);
-
-        if (user == null) {
-            user = super.getUserByIdentity(identity);
-        }
-
-        return user;
-    }
-
-    @Override
-    public Set<Group> getGroups() throws AuthorizationAccessException {
-        final Set<Group> groups = new HashSet<>(configurableUserGroupProvider.getGroups());
-        groups.addAll(super.getGroups());
-        return groups;
-    }
-
-    @Override
-    public Group getGroup(String identifier) throws AuthorizationAccessException {
-        Group group = configurableUserGroupProvider.getGroup(identifier);
-
-        if (group == null) {
-            group = super.getGroup(identifier);
-        }
-
-        return group;
-    }
-
-    @Override
-    public UserAndGroups getUserAndGroups(String identity) throws AuthorizationAccessException {
-        UserAndGroups userAndGroups = configurableUserGroupProvider.getUserAndGroups(identity);
-
-        if (userAndGroups.getUser() == null) {
-            userAndGroups = super.getUserAndGroups(identity);
-        }
-
-        return userAndGroups;
-    }
-
-    @Override
-    public void preDestruction() throws AuthorizerDestructionException {
-        super.preDestruction();
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/CompositeUserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/CompositeUserGroupProvider.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/CompositeUserGroupProvider.java
deleted file mode 100644
index d2f8c4e..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/CompositeUserGroupProvider.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerDestructionException;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-public class CompositeUserGroupProvider implements UserGroupProvider {
-
-    static final String PROP_USER_GROUP_PROVIDER_PREFIX = "User Group Provider ";
-    static final Pattern USER_GROUP_PROVIDER_PATTERN = Pattern.compile(PROP_USER_GROUP_PROVIDER_PREFIX + "\\S+");
-
-    private final boolean allowEmptyProviderList;
-
-    private UserGroupProviderLookup userGroupProviderLookup;
-    private List<UserGroupProvider> userGroupProviders = new ArrayList<>(); // order matters
-
-    public CompositeUserGroupProvider() {
-        this(false);
-    }
-
-    public CompositeUserGroupProvider(boolean allowEmptyProviderList) {
-        this.allowEmptyProviderList = allowEmptyProviderList;
-    }
-
-    @Override
-    public void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
-        userGroupProviderLookup = initializationContext.getUserGroupProviderLookup();
-    }
-
-    @Override
-    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-        for (Map.Entry<String,String> entry : configurationContext.getProperties().entrySet()) {
-            Matcher matcher = USER_GROUP_PROVIDER_PATTERN.matcher(entry.getKey());
-            if (matcher.matches() && !StringUtils.isBlank(entry.getValue())) {
-                final String userGroupProviderKey = entry.getValue();
-                final UserGroupProvider userGroupProvider = userGroupProviderLookup.getUserGroupProvider(userGroupProviderKey);
-
-                if (userGroupProvider == null) {
-                    throw new AuthorizerCreationException(String.format("Unable to locate the configured User Group Provider: %s", userGroupProviderKey));
-                }
-
-                userGroupProviders.add(userGroupProvider);
-            }
-        }
-
-        if (!allowEmptyProviderList && userGroupProviders.isEmpty()) {
-            throw new AuthorizerCreationException("At least one User Group Provider must be configured.");
-        }
-    }
-
-    @Override
-    public Set<User> getUsers() throws AuthorizationAccessException {
-        final Set<User> users = new HashSet<>();
-
-        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
-            users.addAll(userGroupProvider.getUsers());
-        }
-
-        return users;
-    }
-
-    @Override
-    public User getUser(String identifier) throws AuthorizationAccessException {
-        User user = null;
-
-        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
-            user = userGroupProvider.getUser(identifier);
-
-            if (user != null) {
-                break;
-            }
-        }
-
-        return user;
-    }
-
-    @Override
-    public User getUserByIdentity(String identity) throws AuthorizationAccessException {
-        User user = null;
-
-        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
-            user = userGroupProvider.getUserByIdentity(identity);
-
-            if (user != null) {
-                break;
-            }
-        }
-
-        return user;
-    }
-
-    @Override
-    public Set<Group> getGroups() throws AuthorizationAccessException {
-        final Set<Group> groups = new HashSet<>();
-
-        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
-            groups.addAll(userGroupProvider.getGroups());
-        }
-
-        return groups;
-    }
-
-    @Override
-    public Group getGroup(String identifier) throws AuthorizationAccessException {
-        Group group = null;
-
-        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
-            group = userGroupProvider.getGroup(identifier);
-
-            if (group != null) {
-                break;
-            }
-        }
-
-        return group;
-    }
-
-    @Override
-    public UserAndGroups getUserAndGroups(String identity) throws AuthorizationAccessException {
-        UserAndGroups userAndGroups = null;
-
-        for (final UserGroupProvider userGroupProvider : userGroupProviders) {
-            userAndGroups = userGroupProvider.getUserAndGroups(identity);
-
-            if (userAndGroups.getUser() != null) {
-                break;
-            }
-        }
-
-        if (userAndGroups == null) {
-            // per API, returning non null with null user/groups
-            return new UserAndGroups() {
-                @Override
-                public User getUser() {
-                    return null;
-                }
-
-                @Override
-                public Set<Group> getGroups() {
-                    return null;
-                }
-            };
-        } else {
-            // a delegated provider contained a matching user
-            return userAndGroups;
-        }
-    }
-
-    @Override
-    public void preDestruction() throws AuthorizerDestructionException {
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizableLookup.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizableLookup.java
deleted file mode 100644
index 78cd630..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/StandardAuthorizableLookup.java
+++ /dev/null
@@ -1,237 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.authorization.resource.Authorizable;
-import org.apache.nifi.registry.authorization.resource.ResourceFactory;
-import org.apache.nifi.registry.authorization.resource.ResourceType;
-import org.apache.nifi.registry.authorization.resource.AccessPolicyAuthorizable;
-import org.apache.nifi.registry.exception.ResourceNotFoundException;
-
-// TODO, make this spring-wired bean
-public class StandardAuthorizableLookup implements AuthorizableLookup {
-
-    private static final Authorizable TENANTS_AUTHORIZABLE = new Authorizable() {
-        @Override
-        public Authorizable getParentAuthorizable() {
-            return null;
-        }
-
-        @Override
-        public Resource getResource() {
-            return ResourceFactory.getTenantResource();
-        }
-    };
-
-    private static final Authorizable POLICIES_AUTHORIZABLE = new Authorizable() {
-        @Override
-        public Authorizable getParentAuthorizable() {
-            return null;
-        }
-
-        @Override
-        public Resource getResource() {
-            return ResourceFactory.getPoliciesResource();
-        }
-    };
-
-    private static final Authorizable RESOURCES_AUTHORIZABLE = new Authorizable() {
-        @Override
-        public Authorizable getParentAuthorizable() {
-            return null;
-        }
-
-        @Override
-        public Resource getResource() {
-            return ResourceFactory.getResourceResource();
-        }
-    };
-
-    private static final Authorizable BUCKETS_AUTHORIZABLE = new Authorizable() {
-        @Override
-        public Authorizable getParentAuthorizable() {
-            return null;
-        }
-
-        @Override
-        public Resource getResource() {
-            return ResourceFactory.getBucketsResource();
-        }
-    };
-
-    private static final Authorizable PROXY_AUTHORIZABLE = new Authorizable() {
-        @Override
-        public Authorizable getParentAuthorizable() {
-            return null;
-        }
-
-        @Override
-        public Resource getResource() {
-            return ResourceFactory.getProxyResource();
-        }
-    };
-
-    @Override
-    public Authorizable getResourcesAuthorizable() {
-        return RESOURCES_AUTHORIZABLE;
-    }
-
-    @Override
-    public Authorizable getProxyAuthorizable() {
-        return PROXY_AUTHORIZABLE;
-    }
-
-    @Override
-    public Authorizable getTenantsAuthorizable() {
-        return TENANTS_AUTHORIZABLE;
-    }
-
-    @Override
-    public Authorizable getPoliciesAuthorizable() {
-        return POLICIES_AUTHORIZABLE;
-    }
-
-    @Override
-    public Authorizable getBucketsAuthorizable() {
-        return BUCKETS_AUTHORIZABLE;
-    }
-
-    @Override
-    public Authorizable getBucketAuthorizable(String bucketIdentifier) {
-        return new Authorizable() {
-
-            @Override
-            public Authorizable getParentAuthorizable() {
-                return getBucketsAuthorizable();
-            }
-
-            @Override
-            public Resource getResource() {
-                return ResourceFactory.getBucketResource(bucketIdentifier, null);
-            }
-        };
-    }
-
-    @Override
-    public Authorizable getAccessPolicyByResource(final String resource) {
-        try {
-            return new AccessPolicyAuthorizable(getAuthorizableByResource(resource));
-        } catch (final ResourceNotFoundException e) {
-            // the underlying component has been removed or resource is invalid... require /policies permissions
-            return POLICIES_AUTHORIZABLE;
-        }
-    }
-
-    @Override
-    public Authorizable getAuthorizableByResource(String resource) {
-        // parse the resource type
-        ResourceType resourceType = null;
-        for (ResourceType type : ResourceType.values()) {
-            if (resource.equals(type.getValue()) || resource.startsWith(type.getValue() + "/")) {
-                resourceType = type;
-            }
-        }
-
-        if (resourceType == null) {
-            throw new ResourceNotFoundException("Unrecognized resource: " + resource);
-        }
-
-        // if this is a policy resource, there should be another resource type
-        if (ResourceType.Policy.equals(resourceType)) {
-            final ResourceType primaryResourceType = resourceType;
-
-            // get the resource type
-            resource = StringUtils.substringAfter(resource, resourceType.getValue());
-
-            for (ResourceType type : ResourceType.values()) {
-                if (resource.equals(type.getValue()) || resource.startsWith(type.getValue() + "/")) {
-                    resourceType = type;
-                }
-            }
-
-            if (resourceType == null) {
-                throw new ResourceNotFoundException("Unrecognized resource: " + resource);
-            }
-
-            return new AccessPolicyAuthorizable(getAccessPolicy(resourceType, resource));
-        } else {
-            return getAccessPolicy(resourceType, resource);
-        }
-    }
-
-    private Authorizable getAccessPolicy(final ResourceType resourceType, final String resource) {
-        final String slashComponentId = StringUtils.substringAfter(resource, resourceType.getValue());
-        if (slashComponentId.startsWith("/")) {
-            return getAccessPolicyByResource(resourceType, slashComponentId.substring(1));
-        } else {
-            return getAccessPolicyByResource(resourceType);
-        }
-    }
-
-    private Authorizable getAccessPolicyByResource(final ResourceType resourceType, final String childResourceId) {
-        Authorizable authorizable = null;
-        switch (resourceType) {
-            case Bucket:
-                authorizable = getBucketAuthorizable(childResourceId);
-        }
-
-        if (authorizable == null) {
-            throw new IllegalArgumentException("An unexpected type of resource in this policy " + resourceType.getValue());
-        }
-
-        return authorizable;
-    }
-
-    private Authorizable getAccessPolicyByResource(final ResourceType resourceType) {
-        Authorizable authorizable = null;
-        switch (resourceType) {
-
-            case Bucket:
-                authorizable = getBucketsAuthorizable();
-                break;
-            case Policy:
-                authorizable = getPoliciesAuthorizable();
-                break;
-            case Resource:
-                authorizable = new Authorizable() {
-                    @Override
-                    public Authorizable getParentAuthorizable() {
-                        return null;
-                    }
-
-                    @Override
-                    public Resource getResource() {
-                        return ResourceFactory.getResourceResource();
-                    }
-                };
-                break;
-            case Tenant:
-                authorizable = getTenantsAuthorizable();
-                break;
-            case Proxy:
-                authorizable = getProxyAuthorizable();
-        }
-
-        if (authorizable == null) {
-            throw new IllegalArgumentException("An unexpected type of resource in this policy " + resourceType.getValue());
-        }
-
-        return authorizable;
-    }
-
-}


[03/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/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
new file mode 100644
index 0000000..e0eddba
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/NiFiRegistrySecurityConfig.java
@@ -0,0 +1,158 @@
+/*
+ * 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;
+
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.web.security.authentication.NiFiAnonymousUserFilter;
+import org.apache.nifi.registry.web.security.authentication.jwt.JwtAuthenticationFilter;
+import org.apache.nifi.registry.web.security.authentication.jwt.JwtAuthenticationProvider;
+import org.apache.nifi.registry.web.security.authentication.x509.X509AuthenticationFilter;
+import org.apache.nifi.registry.web.security.authentication.x509.X509AuthenticationProvider;
+import org.apache.nifi.registry.web.security.authentication.x509.X509CertificateExtractor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+import org.springframework.security.authentication.AuthenticationManager;
+import org.springframework.security.config.annotation.authentication.builders.AuthenticationManagerBuilder;
+import org.springframework.security.config.annotation.method.configuration.EnableGlobalMethodSecurity;
+import org.springframework.security.config.annotation.web.builders.HttpSecurity;
+import org.springframework.security.config.annotation.web.builders.WebSecurity;
+import org.springframework.security.config.annotation.web.configuration.EnableWebSecurity;
+import org.springframework.security.config.annotation.web.configuration.WebSecurityConfigurerAdapter;
+import org.springframework.security.config.http.SessionCreationPolicy;
+import org.springframework.security.web.authentication.AnonymousAuthenticationFilter;
+import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
+
+/**
+ * NiFi Web Api Spring security
+ */
+@Configuration
+@EnableWebSecurity
+@EnableGlobalMethodSecurity(prePostEnabled = true)
+public class NiFiRegistrySecurityConfig extends WebSecurityConfigurerAdapter {
+    private static final Logger logger = LoggerFactory.getLogger(NiFiRegistrySecurityConfig.class);
+
+    @Autowired private NiFiRegistryProperties properties;
+
+    @Autowired X509CertificateExtractor certificateExtractor;
+    @Autowired X509PrincipalExtractor principalExtractor;
+    @Autowired private X509AuthenticationProvider x509AuthenticationProvider;
+    private X509AuthenticationFilter x509AuthenticationFilter;
+
+    @Autowired private JwtAuthenticationProvider jwtAuthenticationProvider;
+    private JwtAuthenticationFilter jwtAuthenticationFilter;
+
+//    @Autowired private OtpAuthenticationProvider otpAuthenticationProvider;
+//    private OtpAuthenticationFilter otpAuthenticationFilter;
+
+    private NiFiAnonymousUserFilter anonymousAuthenticationFilter;
+
+    public NiFiRegistrySecurityConfig() {
+        super(true); // disable defaults
+    }
+
+    @Override
+    public void configure(WebSecurity webSecurity) throws Exception {
+        // ignore the access endpoints for obtaining the access config, access token
+        // granting, and access status for a given user (note: we are not ignoring the
+        // the /access/download-token endpoints)
+        webSecurity
+                .ignoring()
+                .antMatchers("/access", "/access/config", "/access/token");
+    }
+
+    @Override
+    protected void configure(HttpSecurity http) throws Exception {
+        http
+                .rememberMe().disable()
+                .authorizeRequests()
+                    .anyRequest().fullyAuthenticated()
+                    .and()
+                .sessionManagement()
+                    .sessionCreationPolicy(SessionCreationPolicy.STATELESS);
+
+        // x509
+        http.addFilterBefore(x509AuthenticationFilter(), AnonymousAuthenticationFilter.class);
+
+        // jwt
+        http.addFilterBefore(jwtFilterBean(), AnonymousAuthenticationFilter.class);
+
+        // otp
+        // http.addFilterBefore(otpFilterBean(), AnonymousAuthenticationFilter.class);
+
+        // anonymous
+        http.anonymous().authenticationFilter(anonymousFilter());
+    }
+
+    @Bean
+    @Override
+    public AuthenticationManager authenticationManagerBean() throws Exception {
+        // override xxxBean method so the authentication manager is available in app context (necessary for the method level security)
+        return super.authenticationManagerBean();
+    }
+
+    @Override
+    protected void configure(AuthenticationManagerBuilder auth) throws Exception {
+        auth
+                .authenticationProvider(x509AuthenticationProvider)
+                .authenticationProvider(jwtAuthenticationProvider);
+//                .authenticationProvider(otpAuthenticationProvider); // TODO OTP support
+    }
+
+    @Bean
+    public JwtAuthenticationFilter jwtFilterBean() throws Exception {
+        if (jwtAuthenticationFilter == null) {
+            jwtAuthenticationFilter = new JwtAuthenticationFilter();
+            jwtAuthenticationFilter.setProperties(properties);
+            jwtAuthenticationFilter.setAuthenticationManager(authenticationManager());
+        }
+        return jwtAuthenticationFilter;
+    }
+
+//    @Bean // TODO OtpAuthenticationFilter
+//    public OtpAuthenticationFilter otpFilterBean() throws Exception {
+//        if (otpAuthenticationFilter == null) {
+//            otpAuthenticationFilter = new OtpAuthenticationFilter();
+//            otpAuthenticationFilter.setProperties(properties);
+//            otpAuthenticationFilter.setAuthenticationManager(authenticationManager());
+//        }
+//        return otpAuthenticationFilter;
+//    }
+
+    @Bean
+    public X509AuthenticationFilter x509AuthenticationFilter() throws Exception {
+        if (x509AuthenticationFilter == null) {
+            x509AuthenticationFilter = new X509AuthenticationFilter();
+            x509AuthenticationFilter.setProperties(properties);
+            x509AuthenticationFilter.setCertificateExtractor(certificateExtractor);
+            x509AuthenticationFilter.setPrincipalExtractor(principalExtractor);
+            x509AuthenticationFilter.setAuthenticationManager(authenticationManager());
+        }
+        return x509AuthenticationFilter;
+    }
+
+    @Bean
+    public NiFiAnonymousUserFilter anonymousFilter() throws Exception {
+        if (anonymousAuthenticationFilter == null) {
+            anonymousAuthenticationFilter = new NiFiAnonymousUserFilter();
+        }
+        return anonymousAuthenticationFilter;
+    }
+
+}

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/InvalidAuthenticationException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/InvalidAuthenticationException.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/InvalidAuthenticationException.java
new file mode 100644
index 0000000..016e9cb
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/InvalidAuthenticationException.java
@@ -0,0 +1,35 @@
+/*
+ * 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.exception;
+
+import org.springframework.security.core.AuthenticationException;
+
+/**
+ * Thrown if the authentication of a given request is invalid. For instance,
+ * an expired certificate or token.
+ */
+public class InvalidAuthenticationException extends AuthenticationException {
+
+    public InvalidAuthenticationException(String msg) {
+        super(msg);
+    }
+
+    public InvalidAuthenticationException(String msg, Throwable t) {
+        super(msg, t);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/UntrustedProxyException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/UntrustedProxyException.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/UntrustedProxyException.java
new file mode 100644
index 0000000..6245ac2
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/exception/UntrustedProxyException.java
@@ -0,0 +1,34 @@
+/*
+ * 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.exception;
+
+import org.springframework.security.core.AuthenticationException;
+
+/**
+ *
+ */
+public class UntrustedProxyException extends AuthenticationException {
+
+    public UntrustedProxyException(String msg) {
+        super(msg);
+    }
+
+    public UntrustedProxyException(String msg, Throwable t) {
+        super(msg, t);
+    }
+
+}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationProvider.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationProvider.java
new file mode 100644
index 0000000..2247923
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtAuthenticationProvider.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.jwt;
+
+import io.jsonwebtoken.JwtException;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.user.NiFiUser;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserDetails;
+import org.apache.nifi.registry.security.authorization.user.StandardNiFiUser;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.web.security.authentication.exception.InvalidAuthenticationException;
+import org.apache.nifi.registry.web.security.authentication.NiFiAuthenticationProvider;
+import org.apache.nifi.registry.web.security.authentication.token.NiFiAuthenticationToken;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.AuthenticationException;
+import org.springframework.stereotype.Component;
+
+/**
+ *
+ */
+@Component
+public class JwtAuthenticationProvider extends NiFiAuthenticationProvider {
+
+    private final JwtService jwtService;
+
+    @Autowired
+    public JwtAuthenticationProvider(JwtService jwtService, NiFiRegistryProperties nifiProperties, Authorizer authorizer) {
+        super(nifiProperties, authorizer);
+        this.jwtService = jwtService;
+    }
+
+    @Override
+    public Authentication authenticate(Authentication authentication) throws AuthenticationException {
+        final JwtAuthenticationRequestToken request = (JwtAuthenticationRequestToken) authentication;
+
+        try {
+            final String jwtPrincipal = jwtService.getAuthenticationFromToken(request.getToken());
+            final String mappedIdentity = mapIdentity(jwtPrincipal);
+            final NiFiUser user = new StandardNiFiUser.Builder()
+                    .identity(mappedIdentity)
+                    .groups(getUserGroups(mappedIdentity))
+                    .clientAddress(request.getClientAddress())
+                    .build();
+            return new NiFiAuthenticationToken(new NiFiUserDetails(user));
+        } catch (JwtException e) {
+            throw new InvalidAuthenticationException(e.getMessage(), e);
+        }
+    }
+
+    @Override
+    public boolean supports(Class<?> authentication) {
+        return JwtAuthenticationRequestToken.class.isAssignableFrom(authentication);
+    }
+}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/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
new file mode 100644
index 0000000..49c17ea
--- /dev/null
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/authentication/jwt/JwtService.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.web.security.authentication.jwt;
+
+import io.jsonwebtoken.Claims;
+import io.jsonwebtoken.ExpiredJwtException;
+import io.jsonwebtoken.Jws;
+import io.jsonwebtoken.JwsHeader;
+import io.jsonwebtoken.JwtException;
+import io.jsonwebtoken.Jwts;
+import io.jsonwebtoken.MalformedJwtException;
+import io.jsonwebtoken.SignatureAlgorithm;
+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.key.Key;
+import org.apache.nifi.registry.security.key.KeyService;
+import org.apache.nifi.registry.web.security.authentication.token.LoginAuthenticationToken;
+import org.slf4j.LoggerFactory;
+import org.springframework.stereotype.Service;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Calendar;
+
+@Service
+public class JwtService {
+
+    private static final org.slf4j.Logger logger = LoggerFactory.getLogger(JwtService.class);
+
+    private static final SignatureAlgorithm SIGNATURE_ALGORITHM = SignatureAlgorithm.HS256;
+    private static final String KEY_ID_CLAIM = "kid";
+    private static final String USERNAME_CLAIM = "preferred_username";
+
+    private final KeyService keyService;
+
+    public JwtService(final KeyService keyService) {
+        this.keyService = keyService;
+    }
+
+    public String getAuthenticationFromToken(final String base64EncodedToken) throws JwtException {
+        // The library representations of the JWT should be kept internal to this service.
+        try {
+            final Jws<Claims> jws = parseTokenFromBase64EncodedString(base64EncodedToken);
+
+            if (jws == null) {
+                throw new JwtException("Unable to parse token");
+            }
+
+            // Additional validation that subject is present
+            if (StringUtils.isEmpty(jws.getBody().getSubject())) {
+                throw new JwtException("No subject available in token");
+            }
+
+            // TODO: Validate issuer against active registry?
+            if (StringUtils.isEmpty(jws.getBody().getIssuer())) {
+                throw new JwtException("No issuer available in token");
+            }
+            return jws.getBody().getSubject();
+        } catch (JwtException e) {
+            logger.debug("The Base64 encoded JWT: " + base64EncodedToken);
+            final String errorMessage = "There was an error validating the JWT";
+            logger.error(errorMessage, e);
+            throw e;
+        }
+    }
+
+    private Jws<Claims> parseTokenFromBase64EncodedString(final String base64EncodedToken) throws JwtException {
+        try {
+            return Jwts.parser().setSigningKeyResolver(new SigningKeyResolverAdapter() {
+                @Override
+                public byte[] resolveSigningKeyBytes(JwsHeader header, Claims claims) {
+                    final String identity = claims.getSubject();
+
+                    // Get the key based on the key id in the claims
+                    final String keyId = claims.get(KEY_ID_CLAIM, String.class);
+                    final Key key = keyService.getKey(keyId);
+
+                    // Ensure we were able to find a key that was previously issued by this key service for this user
+                    if (key == null || key.getKey() == null) {
+                        throw new UnsupportedJwtException("Unable to determine signing key for " + identity + " [kid: " + keyId + "]");
+                    }
+
+                    return key.getKey().getBytes(StandardCharsets.UTF_8);
+                }
+            }).parseClaimsJws(base64EncodedToken);
+        } catch (final MalformedJwtException | UnsupportedJwtException | SignatureException | ExpiredJwtException | IllegalArgumentException | AdministrationException 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);
+        }
+    }
+
+    /**
+     * Generates a signed JWT token from the provided (Spring Security) login authentication token.
+     *
+     * @param authenticationToken an instance of the Spring Security token after login credentials have been verified against the respective information source
+     * @return a signed JWT containing the user identity and the identity provider, Base64-encoded
+     * @throws JwtException if there is a problem generating the signed token
+     */
+    public String generateSignedToken(final LoginAuthenticationToken authenticationToken) throws JwtException {
+        if (authenticationToken == null) {
+            throw new IllegalArgumentException("Cannot generate a JWT for a null authentication token");
+        }
+
+        // Set expiration from the token
+        final Calendar expiration = Calendar.getInstance();
+        expiration.setTimeInMillis(authenticationToken.getExpiration());
+
+        final Object principal = authenticationToken.getPrincipal();
+        if (principal == null || StringUtils.isEmpty(principal.toString())) {
+            final String errorMessage = "Cannot generate a JWT for a token with an empty identity issued by " + authenticationToken.getIssuer();
+            logger.error(errorMessage);
+            throw new JwtException(errorMessage);
+        }
+
+        // Create a JWT with the specified authentication
+        final String identity = principal.toString();
+        final String username = authenticationToken.getName();
+
+        try {
+            // Get/create the key for this user
+            final Key key = keyService.getOrCreateKey(identity);
+            final byte[] keyBytes = key.getKey().getBytes(StandardCharsets.UTF_8);
+
+            logger.trace("Generating JWT for " + authenticationToken);
+
+            // TODO: Implement "jti" claim with nonce to prevent replay attacks and allow blacklisting of revoked tokens
+            // Build the token
+            return Jwts.builder().setSubject(identity)
+                    .setIssuer(authenticationToken.getIssuer())
+                    .setAudience(authenticationToken.getIssuer())
+                    .claim(USERNAME_CLAIM, username)
+                    .claim(KEY_ID_CLAIM, key.getId())
+                    .setExpiration(expiration.getTime())
+                    .setIssuedAt(Calendar.getInstance().getTime())
+                    .signWith(SIGNATURE_ALGORITHM, keyBytes).compact();
+        } catch (NullPointerException | AdministrationException e) {
+            final String errorMessage = "Could not retrieve the signing key for JWT for " + identity;
+            logger.error(errorMessage, e);
+            throw new JwtException(errorMessage, e);
+        }
+    }
+}

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


[07/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/UserGroupHolder.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/UserGroupHolder.java b/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/UserGroupHolder.java
deleted file mode 100644
index 3055c19..0000000
--- a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/UserGroupHolder.java
+++ /dev/null
@@ -1,241 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.file;
-
-
-import org.apache.nifi.registry.authorization.file.tenants.generated.Groups;
-import org.apache.nifi.registry.authorization.file.tenants.generated.Tenants;
-import org.apache.nifi.registry.authorization.file.tenants.generated.Users;
-import org.apache.nifi.registry.authorization.Group;
-import org.apache.nifi.registry.authorization.User;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * A holder to provide atomic access to user group data structures.
- */
-public class UserGroupHolder {
-
-    private final Tenants tenants;
-
-    private final Set<User> allUsers;
-    private final Map<String,User> usersById;
-    private final Map<String,User> usersByIdentity;
-
-    private final Set<Group> allGroups;
-    private final Map<String,Group> groupsById;
-    private final Map<String, Set<Group>> groupsByUserIdentity;
-
-    /**
-     * Creates a new holder and populates all convenience data structures.
-     *
-     * @param tenants the current tenants instance
-     */
-    public UserGroupHolder(final Tenants tenants) {
-        this.tenants = tenants;
-
-        // load all users
-        final Users users = tenants.getUsers();
-        final Set<User> allUsers = Collections.unmodifiableSet(createUsers(users));
-
-        // load all groups
-        final Groups groups = tenants.getGroups();
-        final Set<Group> allGroups = Collections.unmodifiableSet(createGroups(groups, users));
-
-        // create a convenience map to retrieve a user by id
-        final Map<String, User> userByIdMap = Collections.unmodifiableMap(createUserByIdMap(allUsers));
-
-        // create a convenience map to retrieve a user by identity
-        final Map<String, User> userByIdentityMap = Collections.unmodifiableMap(createUserByIdentityMap(allUsers));
-
-        // create a convenience map to retrieve a group by id
-        final Map<String, Group> groupByIdMap = Collections.unmodifiableMap(createGroupByIdMap(allGroups));
-
-        // create a convenience map to retrieve the groups for a user identity
-        final Map<String, Set<Group>> groupsByUserIdentityMap = Collections.unmodifiableMap(createGroupsByUserIdentityMap(allGroups, allUsers));
-
-        // set all the holders
-        this.allUsers = allUsers;
-        this.allGroups = allGroups;
-        this.usersById = userByIdMap;
-        this.usersByIdentity = userByIdentityMap;
-        this.groupsById = groupByIdMap;
-        this.groupsByUserIdentity = groupsByUserIdentityMap;
-    }
-
-    /**
-     * Creates a set of Users from the JAXB Users.
-     *
-     * @param users the JAXB Users
-     * @return a set of API Users matching the provided JAXB Users
-     */
-    private Set<User> createUsers(Users users) {
-        Set<User> allUsers = new HashSet<>();
-        if (users == null || users.getUser() == null) {
-            return allUsers;
-        }
-
-        for (org.apache.nifi.registry.authorization.file.tenants.generated.User user : users.getUser()) {
-            final User.Builder builder = new User.Builder()
-                    .identity(user.getIdentity())
-                    .identifier(user.getIdentifier());
-
-            allUsers.add(builder.build());
-        }
-
-        return allUsers;
-    }
-
-    /**
-     * Creates a set of Groups from the JAXB Groups.
-     *
-     * @param groups the JAXB Groups
-     * @return a set of API Groups matching the provided JAXB Groups
-     */
-    private Set<Group> createGroups(Groups groups,
-                                    Users users) {
-        Set<Group> allGroups = new HashSet<>();
-        if (groups == null || groups.getGroup() == null) {
-            return allGroups;
-        }
-
-        for (org.apache.nifi.registry.authorization.file.tenants.generated.Group group : groups.getGroup()) {
-            final Group.Builder builder = new Group.Builder()
-                    .identifier(group.getIdentifier())
-                    .name(group.getName());
-
-            for (org.apache.nifi.registry.authorization.file.tenants.generated.Group.User groupUser : group.getUser()) {
-                builder.addUser(groupUser.getIdentifier());
-            }
-
-            allGroups.add(builder.build());
-        }
-
-        return allGroups;
-    }
-
-    /**
-     * Creates a Map from user identifier to User.
-     *
-     * @param users the set of all users
-     * @return the Map from user identifier to User
-     */
-    private Map<String,User> createUserByIdMap(final Set<User> users) {
-        Map<String,User> usersMap = new HashMap<>();
-        for (User user : users) {
-            usersMap.put(user.getIdentifier(), user);
-        }
-        return usersMap;
-    }
-
-    /**
-     * Creates a Map from user identity to User.
-     *
-     * @param users the set of all users
-     * @return the Map from user identity to User
-     */
-    private Map<String,User> createUserByIdentityMap(final Set<User> users) {
-        Map<String,User> usersMap = new HashMap<>();
-        for (User user : users) {
-            usersMap.put(user.getIdentity(), user);
-        }
-        return usersMap;
-    }
-
-    /**
-     * Creates a Map from group identifier to Group.
-     *
-     * @param groups the set of all groups
-     * @return the Map from group identifier to Group
-     */
-    private Map<String,Group> createGroupByIdMap(final Set<Group> groups) {
-        Map<String,Group> groupsMap = new HashMap<>();
-        for (Group group : groups) {
-            groupsMap.put(group.getIdentifier(), group);
-        }
-        return groupsMap;
-    }
-
-    /**
-     * Creates a Map from user identity to the set of Groups for that identity.
-     *
-     * @param groups all groups
-     * @param users all users
-     * @return a Map from User identity to the set of Groups for that identity
-     */
-    private Map<String, Set<Group>> createGroupsByUserIdentityMap(final Set<Group> groups, final Set<User> users) {
-        Map<String, Set<Group>> groupsByUserIdentity = new HashMap<>();
-
-        for (User user : users) {
-            Set<Group> userGroups = new HashSet<>();
-            for (Group group : groups) {
-                for (String groupUser : group.getUsers()) {
-                    if (groupUser.equals(user.getIdentifier())) {
-                        userGroups.add(group);
-                    }
-                }
-            }
-
-            groupsByUserIdentity.put(user.getIdentity(), userGroups);
-        }
-
-        return groupsByUserIdentity;
-    }
-
-    public Tenants getTenants() {
-        return tenants;
-    }
-
-    public Set<User> getAllUsers() {
-        return allUsers;
-    }
-
-    public Map<String, User> getUsersById() {
-        return usersById;
-    }
-
-    public Map<String, User> getUsersByIdentity() {
-        return usersByIdentity;
-    }
-
-    public Set<Group> getAllGroups() {
-        return allGroups;
-    }
-
-    public Map<String, Group> getGroupsById() {
-        return groupsById;
-    }
-
-    public User getUser(String identity) {
-        if (identity == null) {
-            throw new IllegalArgumentException("Identity cannot be null");
-        }
-        return usersByIdentity.get(identity);
-    }
-
-    public Set<Group> getGroups(String userIdentity) {
-        if (userIdentity == null) {
-            throw new IllegalArgumentException("User Identity cannot be null");
-        }
-        return groupsByUserIdentity.get(userIdentity);
-    }
-
-}

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

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api-impl/src/main/xsd/authorizations.xsd
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/src/main/xsd/authorizations.xsd b/nifi-registry-security-api-impl/src/main/xsd/authorizations.xsd
deleted file mode 100644
index 2c8f805..0000000
--- a/nifi-registry-security-api-impl/src/main/xsd/authorizations.xsd
+++ /dev/null
@@ -1,87 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-      http://www.apache.org/licenses/LICENSE-2.0
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema">
-
-    <xs:complexType name="Policy">
-        <xs:sequence>
-            <xs:element name="group" minOccurs="0" maxOccurs="unbounded" >
-                <xs:complexType>
-                    <xs:attribute name="identifier">
-                        <xs:simpleType>
-                            <xs:restriction base="xs:string">
-                                <xs:minLength value="1"/>
-                                <xs:pattern value=".*[^\s].*"/>
-                            </xs:restriction>
-                        </xs:simpleType>
-                    </xs:attribute>
-                </xs:complexType>
-            </xs:element>
-            <xs:element name="user" minOccurs="0" maxOccurs="unbounded" >
-                <xs:complexType>
-                    <xs:attribute name="identifier">
-                        <xs:simpleType>
-                            <xs:restriction base="xs:string">
-                                <xs:minLength value="1"/>
-                                <xs:pattern value=".*[^\s].*"/>
-                            </xs:restriction>
-                        </xs:simpleType>
-                    </xs:attribute>
-                </xs:complexType>
-            </xs:element>
-        </xs:sequence>
-        <xs:attribute name="identifier">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:minLength value="1"/>
-                    <xs:pattern value=".*[^\s].*"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-        <xs:attribute name="resource">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:minLength value="1"/>
-                    <xs:pattern value=".*[^\s].*"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-        <xs:attribute name="action">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:enumeration value="R"/>
-                    <xs:enumeration value="W"/>
-                    <xs:enumeration value="D"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-    </xs:complexType>
-
-    <xs:complexType name="Policies">
-        <xs:sequence>
-            <xs:element name="policy" type="Policy" minOccurs="0" maxOccurs="unbounded"/>
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- top-level authorizations element -->
-    <xs:element name="authorizations">
-        <xs:complexType>
-            <xs:sequence>
-                <xs:element name="policies" type="Policies" minOccurs="0" maxOccurs="1" />
-            </xs:sequence>
-        </xs:complexType>
-    </xs:element>
-
-</xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api-impl/src/main/xsd/tenants.xsd
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/src/main/xsd/tenants.xsd b/nifi-registry-security-api-impl/src/main/xsd/tenants.xsd
deleted file mode 100644
index c1193c3..0000000
--- a/nifi-registry-security-api-impl/src/main/xsd/tenants.xsd
+++ /dev/null
@@ -1,96 +0,0 @@
-<?xml version="1.0"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-      http://www.apache.org/licenses/LICENSE-2.0
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
--->
-<xs:schema xmlns:xs="http://www.w3.org/2001/XMLSchema">
-
-    <!-- group -->
-    <xs:complexType name="Group">
-        <xs:sequence>
-            <xs:element name="user" minOccurs="0" maxOccurs="unbounded" >
-                <xs:complexType>
-                    <xs:attribute name="identifier">
-                        <xs:simpleType>
-                            <xs:restriction base="xs:string">
-                                <xs:minLength value="1"/>
-                                <xs:pattern value=".*[^\s].*"/>
-                            </xs:restriction>
-                        </xs:simpleType>
-                    </xs:attribute>
-                </xs:complexType>
-            </xs:element>
-        </xs:sequence>
-        <xs:attribute name="identifier">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:minLength value="1"/>
-                    <xs:pattern value=".*[^\s].*"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-        <xs:attribute name="name">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:minLength value="1"/>
-                    <xs:pattern value=".*[^\s].*"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-    </xs:complexType>
-
-    <!-- groups -->
-    <xs:complexType name="Groups">
-        <xs:sequence>
-            <xs:element name="group" type="Group" minOccurs="0" maxOccurs="unbounded"/>
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- user -->
-    <xs:complexType name="User">
-        <xs:attribute name="identifier">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:minLength value="1"/>
-                    <xs:pattern value=".*[^\s].*"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-        <xs:attribute name="identity">
-            <xs:simpleType>
-                <xs:restriction base="xs:string">
-                    <xs:minLength value="1"/>
-                    <xs:pattern value=".*[^\s].*"/>
-                </xs:restriction>
-            </xs:simpleType>
-        </xs:attribute>
-    </xs:complexType>
-
-    <!-- users -->
-    <xs:complexType name="Users">
-        <xs:sequence>
-            <xs:element name="user" type="User" minOccurs="0" maxOccurs="unbounded"/>
-        </xs:sequence>
-    </xs:complexType>
-
-    <!-- top-level authorizations element -->
-    <xs:element name="tenants">
-        <xs:complexType>
-            <xs:sequence>
-                <xs:element name="groups" type="Groups" minOccurs="0" maxOccurs="1" />
-                <xs:element name="users" type="Users" minOccurs="0" maxOccurs="1" />
-            </xs:sequence>
-        </xs:complexType>
-    </xs:element>
-
-</xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AccessPolicy.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AccessPolicy.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AccessPolicy.java
deleted file mode 100644
index 7f2bfa4..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AccessPolicy.java
+++ /dev/null
@@ -1,367 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Objects;
-import java.util.Set;
-import java.util.UUID;
-
-/**
- * Defines a policy for a set of userIdentifiers to perform a set of actions on a given resource.
- */
-public class AccessPolicy {
-
-    private final String identifier;
-
-    private final String resource;
-
-    private final Set<String> users;
-
-    private final Set<String> groups;
-
-    private final RequestAction action;
-
-    private AccessPolicy(final Builder builder) {
-        this.identifier = builder.identifier;
-        this.resource = builder.resource;
-        this.action = builder.action;
-        this.users = Collections.unmodifiableSet(new HashSet<>(builder.users));
-        this.groups = Collections.unmodifiableSet(new HashSet<>(builder.groups));
-
-        if (this.identifier == null || this.identifier.trim().isEmpty()) {
-            throw new IllegalArgumentException("Identifier can not be null or empty");
-        }
-
-        if (this.resource == null) {
-            throw new IllegalArgumentException("Resource can not be null");
-        }
-
-        if (this.action == null) {
-            throw new IllegalArgumentException("Action can not be null");
-        }
-    }
-
-    /**
-     * @return the identifier for this policy
-     */
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    /**
-     * @return the resource for this policy
-     */
-    public String getResource() {
-        return resource;
-    }
-
-    /**
-     * @return an unmodifiable set of user ids for this policy
-     */
-    public Set<String> getUsers() {
-        return users;
-    }
-
-    /**
-     * @return an unmodifiable set of group ids for this policy
-     */
-    public Set<String> getGroups() {
-        return groups;
-    }
-
-    /**
-     * @return the action for this policy
-     */
-    public RequestAction getAction() {
-        return action;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-
-        final AccessPolicy other = (AccessPolicy) obj;
-        return Objects.equals(this.identifier, other.identifier);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(this.identifier);
-    }
-
-    @Override
-    public String toString() {
-        return String.format("identifier[%s], resource[%s], users[%s], groups[%s], action[%s]",
-                getIdentifier(), getResource(), getUsers(), getGroups(), getAction());
-    }
-
-    /**
-     * Builder for Access Policies.
-     */
-    public static class Builder {
-
-        private String identifier;
-        private String resource;
-        private RequestAction action;
-        private Set<String> users = new HashSet<>();
-        private Set<String> groups = new HashSet<>();
-        private final boolean fromPolicy;
-
-        /**
-         * Default constructor for building a new AccessPolicy.
-         */
-        public Builder() {
-            this.fromPolicy = false;
-        }
-
-        /**
-         * Initializes the builder with the state of the provided policy. When using this constructor
-         * the identifier field of the builder can not be changed and will result in an IllegalStateException
-         * if attempting to do so.
-         *
-         * @param other the existing access policy to initialize from
-         */
-        public Builder(final AccessPolicy other) {
-            if (other == null) {
-                throw new IllegalArgumentException("Can not initialize builder with a null access policy");
-            }
-
-            this.identifier = other.getIdentifier();
-            this.resource = other.getResource();
-            this.action = other.getAction();
-            this.users.clear();
-            this.users.addAll(other.getUsers());
-            this.groups.clear();
-            this.groups.addAll(other.getGroups());
-            this.fromPolicy = true;
-        }
-
-        /**
-         * Sets the identifier of the builder.
-         *
-         * @param identifier the identifier to set
-         * @return the builder
-         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Policy
-         */
-        public Builder identifier(final String identifier) {
-            if (fromPolicy) {
-                throw new IllegalStateException(
-                        "Identifier can not be changed when initialized from an existing policy");
-            }
-
-            this.identifier = identifier;
-            return this;
-        }
-
-        /**
-         * Sets the identifier of the builder to a random UUID.
-         *
-         * @return the builder
-         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Policy
-         */
-        public Builder identifierGenerateRandom() {
-            if (fromPolicy) {
-                throw new IllegalStateException(
-                        "Identifier can not be changed when initialized from an existing policy");
-            }
-
-            this.identifier = UUID.randomUUID().toString();
-            return this;
-        }
-
-        /**
-         * Sets the identifier of the builder with a UUID generated from the specified seed string.
-         *
-         * @return the builder
-         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Policy
-         */
-        public Builder identifierGenerateFromSeed(final String seed) {
-            if (fromPolicy) {
-                throw new IllegalStateException(
-                        "Identifier can not be changed when initialized from an existing policy");
-            }
-            if (seed == null) {
-                throw new IllegalArgumentException("Cannot seed the policy identifier with a null value.");
-            }
-
-            this.identifier = UUID.nameUUIDFromBytes(seed.getBytes(StandardCharsets.UTF_8)).toString();
-            return this;
-        }
-
-        /**
-         * Sets the resource of the builder.
-         *
-         * @param resource the resource to set
-         * @return the builder
-         */
-        public Builder resource(final String resource) {
-            this.resource = resource;
-            return this;
-        }
-
-        /**
-         * Adds all the users from the provided set to the builder's set of users.
-         *
-         * @param users the users to add
-         * @return the builder
-         */
-        public Builder addUsers(final Set<String> users) {
-            if (users != null) {
-                this.users.addAll(users);
-            }
-            return this;
-        }
-
-        /**
-         * Adds the given user to the builder's set of users.
-         *
-         * @param user the user to add
-         * @return the builder
-         */
-        public Builder addUser(final String user) {
-            if (user != null) {
-                this.users.add(user);
-            }
-            return this;
-        }
-
-        /**
-         * Removes all users in the provided set from the builder's set of users.
-         *
-         * @param users the users to remove
-         * @return the builder
-         */
-        public Builder removeUsers(final Set<String> users) {
-            if (users != null) {
-                this.users.removeAll(users);
-            }
-            return this;
-        }
-
-        /**
-         * Removes the provided user from the builder's set of users.
-         *
-         * @param user the user to remove
-         * @return the builder
-         */
-        public Builder removeUser(final String user) {
-            if (user != null) {
-                this.users.remove(user);
-            }
-            return this;
-        }
-
-        /**
-         * Clears the builder's set of users so that it is non-null and size == 0.
-         *
-         * @return the builder
-         */
-        public Builder clearUsers() {
-            this.users.clear();
-            return this;
-        }
-
-        /**
-         * Adds all the groups from the provided set to the builder's set of groups.
-         *
-         * @param groups the groups to add
-         * @return the builder
-         */
-        public Builder addGroups(final Set<String> groups) {
-            if (groups != null) {
-                this.groups.addAll(groups);
-            }
-            return this;
-        }
-
-        /**
-         * Adds the given group to the builder's set of groups.
-         *
-         * @param group the group to add
-         * @return the builder
-         */
-        public Builder addGroup(final String group) {
-            if (group != null) {
-                this.groups.add(group);
-            }
-            return this;
-        }
-
-        /**
-         * Removes all groups in the provided set from the builder's set of groups.
-         *
-         * @param groups the groups to remove
-         * @return the builder
-         */
-        public Builder removeGroups(final Set<String> groups) {
-            if (groups != null) {
-                this.groups.removeAll(groups);
-            }
-            return this;
-        }
-
-        /**
-         * Removes the provided groups from the builder's set of groups.
-         *
-         * @param group the group to remove
-         * @return the builder
-         */
-        public Builder removeGroup(final String group) {
-            if (group != null) {
-                this.groups.remove(group);
-            }
-            return this;
-        }
-
-        /**
-         * Clears the builder's set of groups so that it is non-null and size == 0.
-         *
-         * @return the builder
-         */
-        public Builder clearGroups() {
-            this.groups.clear();
-            return this;
-        }
-
-        /**
-         * Sets the action for this builder.
-         *
-         * @param action the action to set
-         * @return the builder
-         */
-        public Builder action(final RequestAction action) {
-            this.action = action;
-            return this;
-        }
-
-        /**
-         * @return a new AccessPolicy constructed from the state of the builder
-         */
-        public AccessPolicy build() {
-            return new AccessPolicy(this);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AccessPolicyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AccessPolicyProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AccessPolicyProvider.java
deleted file mode 100644
index d7e44b2..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AccessPolicyProvider.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerDestructionException;
-
-import java.util.Set;
-
-/**
- * Provides access to AccessPolicies and the configured UserGroupProvider.
- *
- * NOTE: Extensions will be called often and frequently. Because of this, if the underlying implementation needs to
- * make remote calls or expensive calculations those should probably be done asynchronously and/or cache the results.
- *
- * Additionally, extensions need to be thread safe.
- */
-public interface AccessPolicyProvider {
-
-    /**
-     * Retrieves all access policies. Must be non null
-     *
-     * @return a list of policies
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException;
-
-    /**
-     * Retrieves the policy with the given identifier.
-     *
-     * @param identifier the id of the policy to retrieve
-     * @return the policy with the given id, or null if no matching policy exists
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException;
-
-    /**
-     * Gets the access policies for the specified resource identifier and request action.
-     *
-     * @param resourceIdentifier the resource identifier
-     * @param action the request action
-     * @return the policy matching the resouce and action, or null if no matching policy exists
-     * @throws AuthorizationAccessException if there was any unexpected error performing the operation
-     */
-    AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) throws AuthorizationAccessException;
-
-    /**
-     * Returns the UserGroupProvider for this managed Authorizer. Must be non null
-     *
-     * @return the UserGroupProvider
-     */
-    UserGroupProvider getUserGroupProvider();
-
-    /**
-     * Called immediately after instance creation for implementers to perform additional setup
-     *
-     * @param initializationContext in which to initialize
-     */
-    void initialize(AccessPolicyProviderInitializationContext initializationContext) throws AuthorizerCreationException;
-
-    /**
-     * Called to configure the Authorizer.
-     *
-     * @param configurationContext at the time of configuration
-     * @throws AuthorizerCreationException for any issues configuring the provider
-     */
-    void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException;
-
-    /**
-     * Called immediately before instance destruction for implementers to release resources.
-     *
-     * @throws AuthorizerDestructionException If pre-destruction fails.
-     */
-    void preDestruction() throws AuthorizerDestructionException;
-}

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationAuditor.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationAuditor.java
deleted file mode 100644
index 4f017c5..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationAuditor.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-public interface AuthorizationAuditor {
-
-    /**
-     * Audits an authorization request. Will be invoked for any Approved or Denied results. ResourceNotFound
-     * will either re-attempt authorization using a parent resource or will generate a failure result and
-     * audit that.
-     *
-     * @param request the request for authorization
-     * @param result the authorization result
-     */
-    void auditAccessAttempt(final AuthorizationRequest request, final AuthorizationResult result);
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationRequest.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationRequest.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationRequest.java
deleted file mode 100644
index 5c506ef..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationRequest.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.function.Supplier;
-
-/**
- * Represents an authorization request for a given user/entity performing an action against a resource within some userContext.
- */
-public class AuthorizationRequest {
-
-    public static final String DEFAULT_EXPLANATION = "Unable to perform the desired action.";
-
-    private final Resource resource;
-    private final Resource requestedResource;
-    private final String identity;
-    private final Set<String> groups;
-    private final RequestAction action;
-    private final boolean isAccessAttempt;
-    private final boolean isAnonymous;
-    private final Map<String, String> userContext;
-    private final Map<String, String> resourceContext;
-    private final Supplier<String> explanationSupplier;
-
-    private AuthorizationRequest(final Builder builder) {
-        Objects.requireNonNull(builder.resource, "The resource is required when creating an authorization request");
-        Objects.requireNonNull(builder.action, "The action is required when creating an authorization request");
-        Objects.requireNonNull(builder.isAccessAttempt, "Whether this request is an access attempt is request");
-        Objects.requireNonNull(builder.isAnonymous, "Whether this request is being performed by an anonymous user is required");
-
-        this.resource = builder.resource;
-        this.identity = builder.identity;
-        this.groups = builder.groups == null ? null : Collections.unmodifiableSet(builder.groups);
-        this.action = builder.action;
-        this.isAccessAttempt = builder.isAccessAttempt;
-        this.isAnonymous = builder.isAnonymous;
-        this.userContext = builder.userContext == null ? null : Collections.unmodifiableMap(builder.userContext);
-        this.resourceContext = builder.resourceContext == null ? null : Collections.unmodifiableMap(builder.resourceContext);
-        this.explanationSupplier = () -> {
-            final String explanation = builder.explanationSupplier.get();
-
-            // ensure the specified supplier returns non null
-            if (explanation == null) {
-                return DEFAULT_EXPLANATION;
-            } else {
-                return explanation;
-            }
-        };
-
-        if (builder.requestedResource == null) {
-            this.requestedResource = builder.resource;
-        } else {
-            this.requestedResource = builder.requestedResource;
-        }
-    }
-
-    /**
-     * The Resource being authorized. Not null.
-     *
-     * @return The resource
-     */
-    public Resource getResource() {
-        return resource;
-    }
-
-    /**
-     * The original Resource being requested. In cases with inherited policies, this will be a ancestor resource of
-     * of the current resource. The initial request, and cases without inheritance, the requested resource will be
-     * the same as the current resource.
-     *
-     * @return The requested resource
-     */
-    public Resource getRequestedResource() {
-        return requestedResource;
-    }
-
-    /**
-     * The identity accessing the Resource. May be null if the user could not authenticate.
-     *
-     * @return The identity
-     */
-    public String getIdentity() {
-        return identity;
-    }
-
-    /**
-     * The groups the user making this request belongs to. May be null if this NiFi is not configured to load user
-     * groups or empty if the user has no groups
-     *
-     * @return The groups
-     */
-    public Set<String> getGroups() {
-        return groups;
-    }
-
-    /**
-     * Whether this is a direct access attempt of the Resource if if it's being checked as part of another response.
-     *
-     * @return if this is a direct access attempt
-     */
-    public boolean isAccessAttempt() {
-        return isAccessAttempt;
-    }
-
-    /**
-     * Whether the entity accessing is anonymous.
-     *
-     * @return whether the entity is anonymous
-     */
-    public boolean isAnonymous() {
-        return isAnonymous;
-    }
-
-    /**
-     * The action being taken against the Resource. Not null.
-     *
-     * @return The action
-     */
-    public RequestAction getAction() {
-        return action;
-    }
-
-    /**
-     * The userContext of the user request to make additional access decisions. May be null.
-     *
-     * @return  The userContext of the user request
-     */
-    public Map<String, String> getUserContext() {
-        return userContext;
-    }
-
-    /**
-     * The event attributes to make additional access decisions for provenance events. May be null.
-     *
-     * @return  The event attributes
-     */
-    public Map<String, String> getResourceContext() {
-        return resourceContext;
-    }
-
-    /**
-     * A supplier for the explanation if access is denied. Non null.
-     *
-     * @return The explanation supplier if access is denied
-     */
-    public Supplier<String> getExplanationSupplier() {
-        return explanationSupplier;
-    }
-
-    /**
-     * AuthorizationRequest builder.
-     */
-    public static final class Builder {
-
-        private Resource resource;
-        private Resource requestedResource;
-        private String identity;
-        private Set<String> groups;
-        private Boolean isAnonymous;
-        private Boolean isAccessAttempt;
-        private RequestAction action;
-        private Map<String, String> userContext;
-        private Map<String, String> resourceContext;
-        private Supplier<String> explanationSupplier = () -> DEFAULT_EXPLANATION;
-
-        public Builder resource(final Resource resource) {
-            this.resource = resource;
-            return this;
-        }
-
-        public Builder requestedResource(final Resource requestedResource) {
-            this.requestedResource = requestedResource;
-            return this;
-        }
-
-        public Builder identity(final String identity) {
-            this.identity = identity;
-            return this;
-        }
-
-        public Builder groups(final Set<String> groups) {
-            this.groups = groups;
-            return this;
-        }
-
-        public Builder anonymous(final Boolean isAnonymous) {
-            this.isAnonymous = isAnonymous;
-            return this;
-        }
-
-        public Builder accessAttempt(final Boolean isAccessAttempt) {
-            this.isAccessAttempt = isAccessAttempt;
-            return this;
-        }
-
-        public Builder action(final RequestAction action) {
-            this.action = action;
-            return this;
-        }
-
-        public Builder userContext(final Map<String, String> userContext) {
-            if (userContext != null) {
-                this.userContext = new HashMap<>(userContext);
-            }
-            return this;
-        }
-
-        public Builder resourceContext(final Map<String, String> resourceContext) {
-            if (resourceContext != null) {
-                this.resourceContext = new HashMap<>(resourceContext);
-            }
-            return this;
-        }
-
-        public Builder explanationSupplier(final Supplier<String> explanationSupplier) {
-            if (explanationSupplier != null) {
-                this.explanationSupplier = explanationSupplier;
-            }
-            return this;
-        }
-
-        public AuthorizationRequest build() {
-            return new AuthorizationRequest(this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationResult.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationResult.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationResult.java
deleted file mode 100644
index 3cfd056..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/AuthorizationResult.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-/**
- * Represents a decision whether authorization is granted.
- */
-public class AuthorizationResult {
-
-    public enum Result {
-        Approved,
-        Denied,
-        ResourceNotFound
-    }
-
-    private static final AuthorizationResult APPROVED = new AuthorizationResult(Result.Approved, null);
-    private static final AuthorizationResult RESOURCE_NOT_FOUND = new AuthorizationResult(Result.ResourceNotFound, "Not authorized for the requested resource.");
-
-    private final Result result;
-    private final String explanation;
-
-    /**
-     * Creates a new AuthorizationResult with the specified result and explanation.
-     *
-     * @param result of the authorization
-     * @param explanation for the authorization attempt
-     */
-    private AuthorizationResult(Result result, String explanation) {
-        if (Result.Denied.equals(result) && explanation == null) {
-            throw new IllegalArgumentException("An explanation is required when the authorization request is denied.");
-        }
-
-        if (Result.ResourceNotFound.equals(result) && explanation == null) {
-            throw new IllegalArgumentException("An explanation is required when the authorization request is resource not found.");
-        }
-
-        this.result = result;
-        this.explanation = explanation;
-    }
-
-    /**
-     * @return Whether or not the request is approved
-     */
-    public Result getResult() {
-        return result;
-    }
-
-    /**
-     * @return If the request is denied, the reason why. Null otherwise
-     */
-    public String getExplanation() {
-        return explanation;
-    }
-
-    /**
-     * @return a new approved AuthorizationResult
-     */
-    public static AuthorizationResult approved() {
-        return APPROVED;
-    }
-
-    /**
-     * Resource not found will indicate that there are no specific authorization rules for this resource.
-     * @return a new resource not found AuthorizationResult
-     */
-    public static AuthorizationResult resourceNotFound() {
-        return RESOURCE_NOT_FOUND;
-    }
-
-    /**
-     * Creates a new denied AuthorizationResult with a message indicating 'Access is denied'.
-     *
-     * @return a new denied AuthorizationResult
-     */
-    public static AuthorizationResult denied() {
-        return denied(AuthorizationRequest.DEFAULT_EXPLANATION);
-    }
-
-    /**
-     * Creates a new denied AuthorizationResult with the specified explanation.
-     *
-     * @param explanation for why it was denied
-     * @return a new denied AuthorizationResult with the specified explanation
-     * @throws IllegalArgumentException if explanation is null
-     */
-    public static AuthorizationResult denied(String explanation) {
-        return new AuthorizationResult(Result.Denied, explanation);
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Authorizer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Authorizer.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Authorizer.java
deleted file mode 100644
index 3f805d5..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Authorizer.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerDestructionException;
-
-/**
- * Authorizes user requests.
- */
-public interface Authorizer {
-
-    /**
-     * Determines if the specified user/entity is authorized to access the specified resource within the given context.
-     * These details are all contained in the AuthorizationRequest.
-     *
-     * NOTE: This method will be called often and frequently. Because of this, if the underlying implementation needs to
-     * make remote calls or expensive calculations those should probably be done asynchronously and/or cache the results.
-     *
-     * @param   request The authorization request
-     * @return  the authorization result
-     * @throws  AuthorizationAccessException if unable to access the policies
-     */
-    AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException;
-
-    /**
-     * Called immediately after instance creation for implementers to perform additional setup
-     *
-     * @param initializationContext in which to initialize
-     */
-    void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException;
-
-    /**
-     * Called to configure the Authorizer.
-     *
-     * @param configurationContext at the time of configuration
-     * @throws AuthorizerCreationException for any issues configuring the provider
-     */
-    void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException;
-
-    /**
-     * Called immediately before instance destruction for implementers to release resources.
-     *
-     * @throws AuthorizerDestructionException If pre-destruction fails.
-     */
-    void preDestruction() throws AuthorizerDestructionException;
-
-}

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

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ConfigurableAccessPolicyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ConfigurableAccessPolicyProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ConfigurableAccessPolicyProvider.java
deleted file mode 100644
index b563c40..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ConfigurableAccessPolicyProvider.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.UninheritableAuthorizationsException;
-
-/**
- * Provides support for configuring AccessPolicies.
- *
- * NOTE: Extensions will be called often and frequently. Because of this, if the underlying implementation needs to
- * make remote calls or expensive calculations those should probably be done asynchronously and/or cache the results.
- *
- * Additionally, extensions need to be thread safe.
- */
-public interface ConfigurableAccessPolicyProvider extends AccessPolicyProvider {
-
-    /**
-     * Returns a fingerprint representing the authorizations managed by this authorizer. The fingerprint will be
-     * used for comparison to determine if two policy-based authorizers represent a compatible set of policies.
-     *
-     * @return the fingerprint for this Authorizer
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    String getFingerprint() throws AuthorizationAccessException;
-
-    /**
-     * Parses the fingerprint and adds any policies to the current AccessPolicyProvider.
-     *
-     * @param fingerprint the fingerprint that was obtained from calling getFingerprint() on another Authorizer.
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    void inheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
-
-    /**
-     * When the fingerprints are not equal, this method will check if the proposed fingerprint is inheritable.
-     * If the fingerprint is an exact match, this method will not be invoked as there is nothing to inherit.
-     *
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws UninheritableAuthorizationsException if the proposed fingerprint was uninheritable
-     */
-    void checkInheritability(final String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException;
-
-    /**
-     * Adds the given policy ensuring that multiple policies can not be added for the same resource and action.
-     *
-     * @param accessPolicy the policy to add
-     * @return the policy that was added
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    AccessPolicy addAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException;
-
-    /**
-     * Determines whether the specified access policy is configurable. Provides the opportunity for a ConfigurableAccessPolicyProvider to prevent
-     * editing of a specific access policy. By default, all known access policies are configurable.
-     *
-     * @param accessPolicy the access policy
-     * @return is configurable
-     */
-    default boolean isConfigurable(AccessPolicy accessPolicy) {
-        if (accessPolicy == null) {
-            throw new IllegalArgumentException("Access policy cannot be null");
-        }
-
-        return getAccessPolicy(accessPolicy.getIdentifier()) != null;
-    }
-
-    /**
-     * The policy represented by the provided instance will be updated based on the provided instance.
-     *
-     * @param accessPolicy an updated policy
-     * @return the updated policy, or null if no matching policy was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    AccessPolicy updateAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the given policy.
-     *
-     * @param accessPolicy the policy to delete
-     * @return the deleted policy, or null if no matching policy was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    AccessPolicy deleteAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the policy with the specified identifier.
-     *
-     * @param accessPolicyIdentifier the policy to delete
-     * @return the deleted policy, or null if no matching policy was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    AccessPolicy deleteAccessPolicy(String accessPolicyIdentifier) throws AuthorizationAccessException;
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ConfigurableUserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ConfigurableUserGroupProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ConfigurableUserGroupProvider.java
deleted file mode 100644
index 83a7124..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ConfigurableUserGroupProvider.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.UninheritableAuthorizationsException;
-
-/**
- * Provides support for configuring Users and Groups.
- *
- * NOTE: Extensions will be called often and frequently. Because of this, if the underlying implementation needs to
- * make remote calls or expensive calculations those should probably be done asynchronously and/or cache the results.
- *
- * Additionally, extensions need to be thread safe.
- */
-public interface ConfigurableUserGroupProvider extends UserGroupProvider {
-
-    /**
-     * Returns a fingerprint representing the authorizations managed by this authorizer. The fingerprint will be
-     * used for comparison to determine if two policy-based authorizers represent a compatible set of users and/or groups.
-     *
-     * @return the fingerprint for this Authorizer
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    String getFingerprint() throws AuthorizationAccessException;
-
-    /**
-     * Parses the fingerprint and adds any users and groups to the current Authorizer.
-     *
-     * @param fingerprint the fingerprint that was obtained from calling getFingerprint() on another Authorizer.
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    void inheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
-
-    /**
-     * When the fingerprints are not equal, this method will check if the proposed fingerprint is inheritable.
-     * If the fingerprint is an exact match, this method will not be invoked as there is nothing to inherit.
-     *
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws UninheritableAuthorizationsException if the proposed fingerprint was uninheritable
-     */
-    void checkInheritability(final String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException;
-
-    /**
-     * Adds the given user.
-     *
-     * @param user the user to add
-     * @return the user that was added
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws IllegalStateException if there is already a user with the same identity
-     */
-    User addUser(User user) throws AuthorizationAccessException;
-
-    /**
-     * Determines whether the specified user is configurable. Provides the opportunity for a ConfigurableUserGroupProvider to prevent
-     * editing of a specific user. By default, all known users are configurable.
-     *
-     * @param user the user
-     * @return is configurable
-     */
-    default boolean isConfigurable(User user) {
-        if (user == null) {
-            throw new IllegalArgumentException("User cannot be null");
-        }
-
-        return getUser(user.getIdentifier()) != null;
-    }
-
-    /**
-     * The user represented by the provided instance will be updated based on the provided instance.
-     *
-     * @param user an updated user instance
-     * @return the updated user instance, or null if no matching user was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws IllegalStateException if there is already a user with the same identity
-     */
-    User updateUser(final User user) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the given user.
-     *
-     * @param user the user to delete
-     * @return the user that was deleted, or null if no matching user was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    User deleteUser(User user) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the user for the given ID.
-     *
-     * @param userIdentifier the user to delete
-     * @return the user that was deleted, or null if no matching user was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    User deleteUser(String userIdentifier) throws AuthorizationAccessException;
-
-    /**
-     * Adds a new group.
-     *
-     * @param group the Group to add
-     * @return the added Group
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws IllegalStateException if a group with the same name already exists
-     */
-    Group addGroup(Group group) throws AuthorizationAccessException;
-
-    /**
-     * Determines whether the specified group is configurable. Provides the opportunity for a ConfigurableUserGroupProvider to prevent
-     * editing of a specific group. By default, all known groups are configurable.
-     *
-     * @param group the group
-     * @return is configurable
-     */
-    default boolean isConfigurable(Group group) {
-        if (group == null) {
-            throw new IllegalArgumentException("Group cannot be null");
-        }
-
-        return getGroup(group.getIdentifier()) != null;
-    }
-
-    /**
-     * The group represented by the provided instance will be updated based on the provided instance.
-     *
-     * @param group an updated group instance
-     * @return the updated group instance, or null if no matching group was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws IllegalStateException if there is already a group with the same name
-     */
-    Group updateGroup(Group group) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the given group.
-     *
-     * @param group the group to delete
-     * @return the deleted group, or null if no matching group was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    Group deleteGroup(Group group) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the given group.
-     *
-     * @param groupIdentifier the group to delete
-     * @return the deleted group, or null if no matching group was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    Group deleteGroup(String groupIdentifier) throws AuthorizationAccessException;
-}


[09/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/AbstractPolicyBasedAuthorizer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/AbstractPolicyBasedAuthorizer.java b/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/AbstractPolicyBasedAuthorizer.java
deleted file mode 100644
index 718ecc7..0000000
--- a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/AbstractPolicyBasedAuthorizer.java
+++ /dev/null
@@ -1,824 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerDestructionException;
-import org.apache.nifi.registry.authorization.exception.UninheritableAuthorizationsException;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.stream.XMLOutputFactory;
-import javax.xml.stream.XMLStreamException;
-import javax.xml.stream.XMLStreamWriter;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.StringWriter;
-import java.nio.charset.StandardCharsets;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.Set;
-
-/**
- * An Authorizer that provides management of users, groups, and policies.
- */
-public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer {
-
-    static final DocumentBuilderFactory DOCUMENT_BUILDER_FACTORY = DocumentBuilderFactory.newInstance();
-    static final XMLOutputFactory XML_OUTPUT_FACTORY = XMLOutputFactory.newInstance();
-
-    static final String USER_ELEMENT = "user";
-    static final String GROUP_USER_ELEMENT = "groupUser";
-    static final String GROUP_ELEMENT = "group";
-    static final String POLICY_ELEMENT = "policy";
-    static final String POLICY_USER_ELEMENT = "policyUser";
-    static final String POLICY_GROUP_ELEMENT = "policyGroup";
-    static final String IDENTIFIER_ATTR = "identifier";
-    static final String IDENTITY_ATTR = "identity";
-    static final String NAME_ATTR = "name";
-    static final String RESOURCE_ATTR = "resource";
-    static final String ACTIONS_ATTR = "actions";
-
-    @Override
-    public final void onConfigured(final AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-        doOnConfigured(configurationContext);
-    }
-
-    /**
-     * Allows sub-classes to take action when onConfigured is called.
-     *
-     * @param configurationContext the configuration context
-     * @throws AuthorizerCreationException if an error occurs during onConfigured process
-     */
-    protected abstract void doOnConfigured(final AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException;
-
-    @Override
-    public final AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException {
-        final UsersAndAccessPolicies usersAndAccessPolicies = getUsersAndAccessPolicies();
-        final String resourceIdentifier = request.getResource().getIdentifier();
-
-        final AccessPolicy policy = usersAndAccessPolicies.getAccessPolicy(resourceIdentifier, request.getAction());
-        if (policy == null) {
-            return AuthorizationResult.resourceNotFound();
-        }
-
-        final User user = usersAndAccessPolicies.getUser(request.getIdentity());
-        if (user == null) {
-            return AuthorizationResult.denied(String.format("Unknown user with identity '%s'.", request.getIdentity()));
-        }
-
-        final Set<Group> userGroups = usersAndAccessPolicies.getGroups(user.getIdentity());
-        if (policy.getUsers().contains(user.getIdentifier()) || containsGroup(userGroups, policy)) {
-            return AuthorizationResult.approved();
-        }
-
-        return AuthorizationResult.denied(request.getExplanationSupplier().get());
-    }
-
-    /**
-     * Determines if the policy contains one of the user's groups.
-     *
-     * @param userGroups the set of the user's groups
-     * @param policy the policy
-     * @return true if one of the Groups in userGroups is contained in the policy
-     */
-    private boolean containsGroup(final Set<Group> userGroups, final AccessPolicy policy) {
-        if (userGroups.isEmpty() || policy.getGroups().isEmpty()) {
-            return false;
-        }
-
-        for (Group userGroup : userGroups) {
-            if (policy.getGroups().contains(userGroup.getIdentifier())) {
-                return true;
-            }
-        }
-
-        return false;
-    }
-
-    /**
-     * Adds a new group.
-     *
-     * @param group the Group to add
-     * @return the added Group
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws IllegalStateException if a group with the same name already exists
-     */
-    public final synchronized Group addGroup(Group group) throws AuthorizationAccessException {
-        return doAddGroup(group);
-    }
-
-    /**
-     * Adds a new group.
-     *
-     * @param group the Group to add
-     * @return the added Group
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract Group doAddGroup(Group group) throws AuthorizationAccessException;
-
-    /**
-     * Retrieves a Group by id.
-     *
-     * @param identifier the identifier of the Group to retrieve
-     * @return the Group with the given identifier, or null if no matching group was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract Group getGroup(String identifier) throws AuthorizationAccessException;
-
-    /**
-     * The group represented by the provided instance will be updated based on the provided instance.
-     *
-     * @param group an updated group instance
-     * @return the updated group instance, or null if no matching group was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws IllegalStateException if there is already a group with the same name
-     */
-    public final synchronized Group updateGroup(Group group) throws AuthorizationAccessException {
-        return doUpdateGroup(group);
-    }
-
-    /**
-     * The group represented by the provided instance will be updated based on the provided instance.
-     *
-     * @param group an updated group instance
-     * @return the updated group instance, or null if no matching group was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract Group doUpdateGroup(Group group) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the given group.
-     *
-     * @param group the group to delete
-     * @return the deleted group, or null if no matching group was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract Group deleteGroup(Group group) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the group with the given identifier.
-     *
-     * @param groupIdentifier the id of the group to delete
-     * @return the deleted group, or null if no matching group was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract Group deleteGroup(String groupIdentifier) throws AuthorizationAccessException;
-
-    /**
-     * Retrieves all groups.
-     *
-     * @return a list of groups
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract Set<Group> getGroups() throws AuthorizationAccessException;
-
-
-    /**
-     * Adds the given user.
-     *
-     * @param user the user to add
-     * @return the user that was added
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws IllegalStateException if there is already a user with the same identity
-     */
-    public final synchronized User addUser(User user) throws AuthorizationAccessException {
-        return doAddUser(user);
-    }
-
-    /**
-     * Adds the given user.
-     *
-     * @param user the user to add
-     * @return the user that was added
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract User doAddUser(User user) throws AuthorizationAccessException;
-
-    /**
-     * Retrieves the user with the given identifier.
-     *
-     * @param identifier the id of the user to retrieve
-     * @return the user with the given id, or null if no matching user was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract User getUser(String identifier) throws AuthorizationAccessException;
-
-    /**
-     * Retrieves the user with the given identity.
-     *
-     * @param identity the identity of the user to retrieve
-     * @return the user with the given identity, or null if no matching user was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract User getUserByIdentity(String identity) throws AuthorizationAccessException;
-
-    /**
-     * The user represented by the provided instance will be updated based on the provided instance.
-     *
-     * @param user an updated user instance
-     * @return the updated user instance, or null if no matching user was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws IllegalStateException if there is already a user with the same identity
-     */
-    public final synchronized User updateUser(final User user) throws AuthorizationAccessException {
-        return doUpdateUser(user);
-    }
-
-    /**
-     * The user represented by the provided instance will be updated based on the provided instance.
-     *
-     * @param user an updated user instance
-     * @return the updated user instance, or null if no matching user was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract User doUpdateUser(User user) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the given user.
-     *
-     * @param user the user to delete
-     * @return the user that was deleted, or null if no matching user was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract User deleteUser(User user) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the user with the given id.
-     *
-     * @param userIdentifier the identifier of the user to delete
-     * @return the user that was deleted, or null if no matching user was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract User deleteUser(String userIdentifier) throws AuthorizationAccessException;
-
-    /**
-     * Retrieves all users.
-     *
-     * @return a list of users
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract Set<User> getUsers() throws AuthorizationAccessException;
-
-    /**
-     * Adds the given policy ensuring that multiple policies can not be added for the same resource and action.
-     *
-     * @param accessPolicy the policy to add
-     * @return the policy that was added
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public final synchronized AccessPolicy addAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
-        return doAddAccessPolicy(accessPolicy);
-    }
-
-    /**
-     * Adds the given policy.
-     *
-     * @param accessPolicy the policy to add
-     * @return the policy that was added
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    protected abstract AccessPolicy doAddAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException;
-
-    /**
-     * Retrieves the policy with the given identifier.
-     *
-     * @param identifier the id of the policy to retrieve
-     * @return the policy with the given id, or null if no matching policy exists
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException;
-
-    /**
-     * The policy represented by the provided instance will be updated based on the provided instance.
-     *
-     * @param accessPolicy an updated policy
-     * @return the updated policy, or null if no matching policy was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract AccessPolicy updateAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the given policy.
-     *
-     * @param policy the policy to delete
-     * @return the deleted policy, or null if no matching policy was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract AccessPolicy deleteAccessPolicy(AccessPolicy policy) throws AuthorizationAccessException;
-
-    /**
-     * Deletes the policy with the given id.
-     *
-     * @param policyIdentifier the id of the policy to delete
-     * @return the deleted policy, or null if no matching policy was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract AccessPolicy deleteAccessPolicy(String policyIdentifier) throws AuthorizationAccessException;
-
-    /**
-     * Retrieves all access policies.
-     *
-     * @return a list of policies
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException;
-
-    /**
-     * Returns the UserAccessPolicies instance.
-     *
-     * @return the UserAccessPolicies instance
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    public abstract UsersAndAccessPolicies getUsersAndAccessPolicies() throws AuthorizationAccessException;
-
-    /**
-     * Returns whether the proposed fingerprint is inheritable.
-     *
-     * @param proposedFingerprint the proposed fingerprint
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws UninheritableAuthorizationsException if the proposed fingerprint was uninheritable
-     */
-    @Override
-    public final void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
-        try {
-            // ensure we understand the proposed fingerprint
-            parsePoliciesUsersAndGroups(proposedFingerprint);
-        } catch (final AuthorizationAccessException e) {
-            throw new UninheritableAuthorizationsException("Unable to parse proposed fingerprint: " + e);
-        }
-
-        final List<User> users = getSortedUsers();
-        final List<Group> groups = getSortedGroups();
-        final List<AccessPolicy> accessPolicies = getSortedAccessPolicies();
-
-        // ensure we're in a state to inherit
-        if (!users.isEmpty() || !groups.isEmpty() || !accessPolicies.isEmpty()) {
-            throw new UninheritableAuthorizationsException("Proposed fingerprint is not inheritable because the current Authorizations is not empty..");
-        }
-    }
-
-    /**
-     * Parses the fingerprint and adds any users, groups, and policies to the current Authorizer.
-     *
-     * @param fingerprint the fingerprint that was obtained from calling getFingerprint() on another Authorizer.
-     */
-    @Override
-    public final void inheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
-        if (fingerprint == null || fingerprint.trim().isEmpty()) {
-            return;
-        }
-
-        final PoliciesUsersAndGroups policiesUsersAndGroups = parsePoliciesUsersAndGroups(fingerprint);
-        policiesUsersAndGroups.getUsers().forEach(user -> addUser(user));
-        policiesUsersAndGroups.getGroups().forEach(group -> addGroup(group));
-        policiesUsersAndGroups.getAccessPolicies().forEach(policy -> addAccessPolicy(policy));
-    }
-
-    private PoliciesUsersAndGroups parsePoliciesUsersAndGroups(final String fingerprint) {
-        final List<AccessPolicy> accessPolicies = new ArrayList<>();
-        final List<User> users = new ArrayList<>();
-        final List<Group> groups = new ArrayList<>();
-
-        final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
-        try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
-            final DocumentBuilder docBuilder = DOCUMENT_BUILDER_FACTORY.newDocumentBuilder();
-            final Document document = docBuilder.parse(in);
-            final Element rootElement = document.getDocumentElement();
-
-            // parse all the users and add them to the current authorizer
-            NodeList userNodes = rootElement.getElementsByTagName(USER_ELEMENT);
-            for (int i=0; i < userNodes.getLength(); i++) {
-                Node userNode = userNodes.item(i);
-                users.add(parseUser((Element) userNode));
-            }
-
-            // parse all the groups and add them to the current authorizer
-            NodeList groupNodes = rootElement.getElementsByTagName(GROUP_ELEMENT);
-            for (int i=0; i < groupNodes.getLength(); i++) {
-                Node groupNode = groupNodes.item(i);
-                groups.add(parseGroup((Element) groupNode));
-            }
-
-            // parse all the policies and add them to the current authorizer
-            NodeList policyNodes = rootElement.getElementsByTagName(POLICY_ELEMENT);
-            for (int i=0; i < policyNodes.getLength(); i++) {
-                Node policyNode = policyNodes.item(i);
-                accessPolicies.add(parsePolicy((Element) policyNode));
-            }
-        } catch (SAXException | ParserConfigurationException | IOException e) {
-            throw new AuthorizationAccessException("Unable to parse fingerprint", e);
-        }
-
-        return new PoliciesUsersAndGroups(accessPolicies, users, groups);
-    }
-
-    private User parseUser(final Element element) {
-        final User.Builder builder = new User.Builder()
-                .identifier(element.getAttribute(IDENTIFIER_ATTR))
-                .identity(element.getAttribute(IDENTITY_ATTR));
-
-        return builder.build();
-    }
-
-    private Group parseGroup(final Element element) {
-        final Group.Builder builder = new Group.Builder()
-                .identifier(element.getAttribute(IDENTIFIER_ATTR))
-                .name(element.getAttribute(NAME_ATTR));
-
-        NodeList groupUsers = element.getElementsByTagName(GROUP_USER_ELEMENT);
-        for (int i=0; i < groupUsers.getLength(); i++) {
-            Element groupUserNode = (Element) groupUsers.item(i);
-            builder.addUser(groupUserNode.getAttribute(IDENTIFIER_ATTR));
-        }
-
-        return builder.build();
-    }
-
-    private AccessPolicy parsePolicy(final Element element) {
-        final AccessPolicy.Builder builder = new AccessPolicy.Builder()
-                .identifier(element.getAttribute(IDENTIFIER_ATTR))
-                .resource(element.getAttribute(RESOURCE_ATTR));
-
-        final String actions = element.getAttribute(ACTIONS_ATTR);
-        if (actions.equals(RequestAction.READ.name())) {
-            builder.action(RequestAction.READ);
-        } else if (actions.equals(RequestAction.WRITE.name())) {
-            builder.action(RequestAction.WRITE);
-        } else if (actions.equals(RequestAction.DELETE.name())) {
-            builder.action(RequestAction.DELETE);
-        } else {
-            throw new IllegalStateException("Unknown Policy Action: " + actions);
-        }
-
-        NodeList policyUsers = element.getElementsByTagName(POLICY_USER_ELEMENT);
-        for (int i=0; i < policyUsers.getLength(); i++) {
-            Element policyUserNode = (Element) policyUsers.item(i);
-            builder.addUser(policyUserNode.getAttribute(IDENTIFIER_ATTR));
-        }
-
-        NodeList policyGroups = element.getElementsByTagName(POLICY_GROUP_ELEMENT);
-        for (int i=0; i < policyGroups.getLength(); i++) {
-            Element policyGroupNode = (Element) policyGroups.item(i);
-            builder.addGroup(policyGroupNode.getAttribute(IDENTIFIER_ATTR));
-        }
-
-        return builder.build();
-    }
-
-    @Override
-    public final AccessPolicyProvider getAccessPolicyProvider() {
-        return new ConfigurableAccessPolicyProvider() {
-            @Override
-            public Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException {
-                return AbstractPolicyBasedAuthorizer.this.getAccessPolicies();
-            }
-
-            @Override
-            public AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException {
-                return AbstractPolicyBasedAuthorizer.this.getAccessPolicy(identifier);
-            }
-
-            @Override
-            public AccessPolicy addAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
-                return AbstractPolicyBasedAuthorizer.this.addAccessPolicy(accessPolicy);
-            }
-
-            @Override
-            public AccessPolicy updateAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
-                return AbstractPolicyBasedAuthorizer.this.updateAccessPolicy(accessPolicy);
-            }
-
-            @Override
-            public AccessPolicy deleteAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
-                return AbstractPolicyBasedAuthorizer.this.deleteAccessPolicy(accessPolicy);
-            }
-
-            @Override
-            public AccessPolicy deleteAccessPolicy(String accessPolicyIdentifier) throws AuthorizationAccessException {
-                return AbstractPolicyBasedAuthorizer.this.deleteAccessPolicy(accessPolicyIdentifier);
-            }
-
-            @Override
-            public AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) throws AuthorizationAccessException {
-                final UsersAndAccessPolicies usersAndAccessPolicies = AbstractPolicyBasedAuthorizer.this.getUsersAndAccessPolicies();
-                return usersAndAccessPolicies.getAccessPolicy(resourceIdentifier, action);
-            }
-
-            @Override
-            public String getFingerprint() throws AuthorizationAccessException {
-                // fingerprint is managed by the encapsulating class
-                throw new UnsupportedOperationException();
-            }
-
-            @Override
-            public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
-                // fingerprint is managed by the encapsulating class
-                throw new UnsupportedOperationException();
-            }
-
-            @Override
-            public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
-                // fingerprint is managed by the encapsulating class
-                throw new UnsupportedOperationException();
-            }
-
-            @Override
-            public UserGroupProvider getUserGroupProvider() {
-                return new ConfigurableUserGroupProvider() {
-                    @Override
-                    public User addUser(User user) throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.addUser(user);
-                    }
-
-                    @Override
-                    public User updateUser(User user) throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.updateUser(user);
-                    }
-
-                    @Override
-                    public User deleteUser(User user) throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.deleteUser(user);
-                    }
-
-                    @Override
-                    public User deleteUser(String userIdentifier) throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.deleteUser(userIdentifier);
-                    }
-
-                    @Override
-                    public Group addGroup(Group group) throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.addGroup(group);
-                    }
-
-                    @Override
-                    public Group updateGroup(Group group) throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.updateGroup(group);
-                    }
-
-                    @Override
-                    public Group deleteGroup(Group group) throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.deleteGroup(group);
-                    }
-
-                    @Override
-                    public Group deleteGroup(String groupIdentifier) throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.deleteGroup(groupIdentifier);
-                    }
-
-                    @Override
-                    public Set<User> getUsers() throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.getUsers();
-                    }
-
-                    @Override
-                    public User getUser(String identifier) throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.getUser(identifier);
-                    }
-
-                    @Override
-                    public User getUserByIdentity(String identity) throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.getUserByIdentity(identity);
-                    }
-
-                    @Override
-                    public Set<Group> getGroups() throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.getGroups();
-                    }
-
-                    @Override
-                    public Group getGroup(String identifier) throws AuthorizationAccessException {
-                        return AbstractPolicyBasedAuthorizer.this.getGroup(identifier);
-                    }
-
-                    @Override
-                    public UserAndGroups getUserAndGroups(String identity) throws AuthorizationAccessException {
-                        final UsersAndAccessPolicies usersAndAccessPolicies = AbstractPolicyBasedAuthorizer.this.getUsersAndAccessPolicies();
-                        final User user = usersAndAccessPolicies.getUser(identity);
-                        final Set<Group> groups = usersAndAccessPolicies.getGroups(identity);
-
-                        return new UserAndGroups() {
-                            @Override
-                            public User getUser() {
-                                return user;
-                            }
-
-                            @Override
-                            public Set<Group> getGroups() {
-                                return groups;
-                            }
-                        };
-                    }
-
-                    @Override
-                    public String getFingerprint() throws AuthorizationAccessException {
-                        // fingerprint is managed by the encapsulating class
-                        throw new UnsupportedOperationException();
-                    }
-
-                    @Override
-                    public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
-                        // fingerprint is managed by the encapsulating class
-                        throw new UnsupportedOperationException();
-                    }
-
-                    @Override
-                    public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
-                        // fingerprint is managed by the encapsulating class
-                        throw new UnsupportedOperationException();
-                    }
-
-                    @Override
-                    public void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
-                    }
-
-                    @Override
-                    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-                    }
-
-                    @Override
-                    public void preDestruction() throws AuthorizerDestructionException {
-                    }
-                };
-            }
-
-            @Override
-            public void initialize(AccessPolicyProviderInitializationContext initializationContext) throws AuthorizerCreationException {
-            }
-
-            @Override
-            public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
-            }
-
-            @Override
-            public void preDestruction() throws AuthorizerDestructionException {
-            }
-        };
-    }
-
-    /**
-     * Returns a fingerprint representing the authorizations managed by this authorizer. The fingerprint will be
-     * used for comparison to determine if two policy-based authorizers represent a compatible set of users,
-     * groups, and policies.
-     *
-     * @return the fingerprint for this Authorizer
-     */
-    @Override
-    public final String getFingerprint() throws AuthorizationAccessException {
-        final List<User> users = getSortedUsers();
-        final List<Group> groups = getSortedGroups();
-        final List<AccessPolicy> policies = getSortedAccessPolicies();
-
-        XMLStreamWriter writer = null;
-        final StringWriter out = new StringWriter();
-        try {
-            writer = XML_OUTPUT_FACTORY.createXMLStreamWriter(out);
-            writer.writeStartDocument();
-            writer.writeStartElement("authorizations");
-
-            for (User user : users) {
-                writeUser(writer, user);
-            }
-            for (Group group : groups) {
-                writeGroup(writer, group);
-            }
-            for (AccessPolicy policy : policies) {
-                writePolicy(writer, policy);
-            }
-
-            writer.writeEndElement();
-            writer.writeEndDocument();
-            writer.flush();
-        } catch (XMLStreamException e) {
-            throw new AuthorizationAccessException("Unable to generate fingerprint", e);
-        } finally {
-            if (writer != null) {
-                try {
-                    writer.close();
-                } catch (XMLStreamException e) {
-                    // nothing to do here
-                }
-            }
-        }
-
-        return out.toString();
-    }
-
-    private void writeUser(final XMLStreamWriter writer, final User user) throws XMLStreamException {
-        writer.writeStartElement(USER_ELEMENT);
-        writer.writeAttribute(IDENTIFIER_ATTR, user.getIdentifier());
-        writer.writeAttribute(IDENTITY_ATTR, user.getIdentity());
-        writer.writeEndElement();
-    }
-
-    private void writeGroup(final XMLStreamWriter writer, final Group group) throws XMLStreamException {
-        List<String> users = new ArrayList<>(group.getUsers());
-        Collections.sort(users);
-
-        writer.writeStartElement(GROUP_ELEMENT);
-        writer.writeAttribute(IDENTIFIER_ATTR, group.getIdentifier());
-        writer.writeAttribute(NAME_ATTR, group.getName());
-
-        for (String user : users) {
-            writer.writeStartElement(GROUP_USER_ELEMENT);
-            writer.writeAttribute(IDENTIFIER_ATTR, user);
-            writer.writeEndElement();
-        }
-
-        writer.writeEndElement();
-    }
-
-    private void writePolicy(final XMLStreamWriter writer, final AccessPolicy policy) throws XMLStreamException {
-        // sort the users for the policy
-        List<String> policyUsers = new ArrayList<>(policy.getUsers());
-        Collections.sort(policyUsers);
-
-        // sort the groups for this policy
-        List<String> policyGroups = new ArrayList<>(policy.getGroups());
-        Collections.sort(policyGroups);
-
-        writer.writeStartElement(POLICY_ELEMENT);
-        writer.writeAttribute(IDENTIFIER_ATTR, policy.getIdentifier());
-        writer.writeAttribute(RESOURCE_ATTR, policy.getResource());
-        writer.writeAttribute(ACTIONS_ATTR, policy.getAction().name());
-
-        for (String policyUser : policyUsers) {
-            writer.writeStartElement(POLICY_USER_ELEMENT);
-            writer.writeAttribute(IDENTIFIER_ATTR, policyUser);
-            writer.writeEndElement();
-        }
-
-        for (String policyGroup : policyGroups) {
-            writer.writeStartElement(POLICY_GROUP_ELEMENT);
-            writer.writeAttribute(IDENTIFIER_ATTR, policyGroup);
-            writer.writeEndElement();
-        }
-
-        writer.writeEndElement();
-    }
-
-    private List<AccessPolicy> getSortedAccessPolicies() {
-        final List<AccessPolicy> policies = new ArrayList<>(getAccessPolicies());
-        Collections.sort(policies, Comparator.comparing(AccessPolicy::getIdentifier));
-        return policies;
-    }
-
-    private List<Group> getSortedGroups() {
-        final List<Group> groups = new ArrayList<>(getGroups());
-        Collections.sort(groups, Comparator.comparing(Group::getIdentifier));
-        return groups;
-    }
-
-    private List<User> getSortedUsers() {
-        final List<User> users = new ArrayList<>(getUsers());
-        Collections.sort(users, Comparator.comparing(User::getIdentifier));
-        return users;
-    }
-
-    private static class PoliciesUsersAndGroups {
-        final List<AccessPolicy> accessPolicies;
-        final List<User> users;
-        final List<Group> groups;
-
-        public PoliciesUsersAndGroups(List<AccessPolicy> accessPolicies, List<User> users, List<Group> groups) {
-            this.accessPolicies = accessPolicies;
-            this.users = users;
-            this.groups = groups;
-        }
-
-        public List<AccessPolicy> getAccessPolicies() {
-            return accessPolicies;
-        }
-
-        public List<User> getUsers() {
-            return users;
-        }
-
-        public List<Group> getGroups() {
-            return groups;
-        }
-    }
-}

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/UsersAndAccessPolicies.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/UsersAndAccessPolicies.java b/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/UsersAndAccessPolicies.java
deleted file mode 100644
index 7e05c9c..0000000
--- a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/UsersAndAccessPolicies.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import java.util.Set;
-
-/**
- * A holder object to provide atomic access to policies for a given resource and users by
- * identity. Implementations must ensure consistent access to the data backing this instance.
- */
-public interface UsersAndAccessPolicies {
-
-    /**
-     * Retrieves the set of access policies for a given resource and action.
-     *
-     * @param resourceIdentifier the resource identifier to retrieve policies for
-     * @param action the action to retrieve policies for
-     * @return the access policy for the given resource and action
-     */
-    AccessPolicy getAccessPolicy(final String resourceIdentifier, final RequestAction action);
-
-    /**
-     * Retrieves a user by an identity string.
-     *
-     * @param identity the identity of the user to retrieve
-     * @return the user with the given identity
-     */
-    User getUser(final String identity);
-
-    /**
-     * Retrieves the groups for a given user identity.
-     *
-     * @param userIdentity a user identity
-     * @return the set of groups for the given user identity
-     */
-    Set<Group> getGroups(final String userIdentity);
-
-}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/AuthorizationsHolder.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/AuthorizationsHolder.java b/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/AuthorizationsHolder.java
deleted file mode 100644
index 38a1571..0000000
--- a/nifi-registry-security-api-impl/src/main/java/org/apache/nifi/registry/authorization/file/AuthorizationsHolder.java
+++ /dev/null
@@ -1,187 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.file;
-
-
-import org.apache.nifi.registry.authorization.file.generated.Authorizations;
-import org.apache.nifi.registry.authorization.file.generated.Policies;
-import org.apache.nifi.registry.authorization.AccessPolicy;
-import org.apache.nifi.registry.authorization.RequestAction;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * A holder to provide atomic access to data structures.
- */
-public class AuthorizationsHolder {
-
-    private final Authorizations authorizations;
-
-    private final Set<AccessPolicy> allPolicies;
-    private final Map<String, Set<AccessPolicy>> policiesByResource;
-    private final Map<String, AccessPolicy> policiesById;
-
-    /**
-     * Creates a new holder and populates all convenience authorizations data structures.
-     *
-     * @param authorizations the current authorizations instance
-     */
-    public AuthorizationsHolder(final Authorizations authorizations) {
-        this.authorizations = authorizations;
-
-        // load all access policies
-        final Policies policies = authorizations.getPolicies();
-        final Set<AccessPolicy> allPolicies = Collections.unmodifiableSet(createAccessPolicies(policies));
-
-        // create a convenience map from resource id to policies
-        final Map<String, Set<AccessPolicy>> policiesByResourceMap = Collections.unmodifiableMap(createResourcePolicyMap(allPolicies));
-
-        // create a convenience map from policy id to policy
-        final Map<String, AccessPolicy> policiesByIdMap = Collections.unmodifiableMap(createPoliciesByIdMap(allPolicies));
-
-        // set all the holders
-        this.allPolicies = allPolicies;
-        this.policiesByResource = policiesByResourceMap;
-        this.policiesById = policiesByIdMap;
-    }
-
-    /**
-     * Creates AccessPolicies from the JAXB Policies.
-     *
-     * @param policies the JAXB Policies element
-     * @return a set of AccessPolicies corresponding to the provided Resources
-     */
-    private Set<AccessPolicy> createAccessPolicies(org.apache.nifi.registry.authorization.file.generated.Policies policies) {
-        Set<AccessPolicy> allPolicies = new HashSet<>();
-        if (policies == null || policies.getPolicy() == null) {
-            return allPolicies;
-        }
-
-        // load the new authorizations
-        for (final org.apache.nifi.registry.authorization.file.generated.Policy policy : policies.getPolicy()) {
-            final String policyIdentifier = policy.getIdentifier();
-            final String resourceIdentifier = policy.getResource();
-
-            // start a new builder and set the policy and resource identifiers
-            final AccessPolicy.Builder builder = new AccessPolicy.Builder()
-                    .identifier(policyIdentifier)
-                    .resource(resourceIdentifier);
-
-            // add each user identifier
-            for (org.apache.nifi.registry.authorization.file.generated.Policy.User user : policy.getUser()) {
-                builder.addUser(user.getIdentifier());
-            }
-
-            // add each group identifier
-            for (org.apache.nifi.registry.authorization.file.generated.Policy.Group group : policy.getGroup()) {
-                builder.addGroup(group.getIdentifier());
-            }
-
-            // add the appropriate request actions
-            final String authorizationCode = policy.getAction();
-            if (authorizationCode.equals(FileAccessPolicyProvider.READ_CODE)) {
-                builder.action(RequestAction.READ);
-            } else if (authorizationCode.equals(FileAccessPolicyProvider.WRITE_CODE)){
-                builder.action(RequestAction.WRITE);
-            } else if (authorizationCode.equals(FileAccessPolicyProvider.DELETE_CODE)){
-                builder.action(RequestAction.DELETE);
-            } else {
-                throw new IllegalStateException("Unknown Policy Action: " + authorizationCode);
-            }
-
-            // build the policy and add it to the map
-            allPolicies.add(builder.build());
-        }
-
-        return allPolicies;
-    }
-
-    /**
-     * Creates a map from resource identifier to the set of policies for the given resource.
-     *
-     * @param allPolicies the set of all policies
-     * @return a map from resource identifier to policies
-     */
-    private Map<String, Set<AccessPolicy>> createResourcePolicyMap(final Set<AccessPolicy> allPolicies) {
-        Map<String, Set<AccessPolicy>> resourcePolicies = new HashMap<>();
-
-        for (AccessPolicy policy : allPolicies) {
-            Set<AccessPolicy> policies = resourcePolicies.get(policy.getResource());
-            if (policies == null) {
-                policies = new HashSet<>();
-                resourcePolicies.put(policy.getResource(), policies);
-            }
-            policies.add(policy);
-        }
-
-        return resourcePolicies;
-    }
-
-    /**
-     * Creates a Map from policy identifier to AccessPolicy.
-     *
-     * @param policies the set of all access policies
-     * @return the Map from policy identifier to AccessPolicy
-     */
-    private Map<String, AccessPolicy> createPoliciesByIdMap(final Set<AccessPolicy> policies) {
-        Map<String,AccessPolicy> policyMap = new HashMap<>();
-        for (AccessPolicy policy : policies) {
-            policyMap.put(policy.getIdentifier(), policy);
-        }
-        return policyMap;
-    }
-
-    public Authorizations getAuthorizations() {
-        return authorizations;
-    }
-
-    public Set<AccessPolicy> getAllPolicies() {
-        return allPolicies;
-    }
-
-    public Map<String, Set<AccessPolicy>> getPoliciesByResource() {
-        return policiesByResource;
-    }
-
-    public Map<String, AccessPolicy> getPoliciesById() {
-        return policiesById;
-    }
-
-    public AccessPolicy getAccessPolicy(final String resourceIdentifier, final RequestAction action) {
-        if (resourceIdentifier == null) {
-            throw new IllegalArgumentException("Resource Identifier cannot be null");
-        }
-
-        final Set<AccessPolicy> resourcePolicies = policiesByResource.get(resourceIdentifier);
-        if (resourcePolicies == null) {
-            return null;
-        }
-
-        for (AccessPolicy accessPolicy : resourcePolicies) {
-            if (accessPolicy.getAction() == action) {
-                return accessPolicy;
-            }
-        }
-
-        return null;
-    }
-
-}


[13/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardManagedAuthorizer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardManagedAuthorizer.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardManagedAuthorizer.java
new file mode 100644
index 0000000..b6f9842
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/StandardManagedAuthorizer.java
@@ -0,0 +1,278 @@
+/*
+ * 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.authorization;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.security.authorization.AccessPolicy;
+import org.apache.nifi.registry.security.authorization.AccessPolicyProvider;
+import org.apache.nifi.registry.security.authorization.AccessPolicyProviderLookup;
+import org.apache.nifi.registry.security.authorization.AuthorizationRequest;
+import org.apache.nifi.registry.security.authorization.AuthorizationResult;
+import org.apache.nifi.registry.security.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.registry.security.authorization.AuthorizerInitializationContext;
+import org.apache.nifi.registry.security.authorization.ConfigurableAccessPolicyProvider;
+import org.apache.nifi.registry.security.authorization.ConfigurableUserGroupProvider;
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.ManagedAuthorizer;
+import org.apache.nifi.registry.security.authorization.User;
+import org.apache.nifi.registry.security.authorization.UserAndGroups;
+import org.apache.nifi.registry.security.authorization.UserGroupProvider;
+import org.apache.nifi.registry.security.authorization.exception.UninheritableAuthorizationsException;
+import org.apache.nifi.registry.util.PropertyValue;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamWriter;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Set;
+
+public class StandardManagedAuthorizer implements ManagedAuthorizer {
+
+    private static final DocumentBuilderFactory DOCUMENT_BUILDER_FACTORY = DocumentBuilderFactory.newInstance();
+    private static final XMLOutputFactory XML_OUTPUT_FACTORY = XMLOutputFactory.newInstance();
+
+    private static final String USER_GROUP_PROVIDER_ELEMENT = "userGroupProvider";
+    private static final String ACCESS_POLICY_PROVIDER_ELEMENT = "accessPolicyProvider";
+
+    private AccessPolicyProviderLookup accessPolicyProviderLookup;
+    private AccessPolicyProvider accessPolicyProvider;
+    private UserGroupProvider userGroupProvider;
+
+    public StandardManagedAuthorizer() {}
+
+    // exposed for testing to inject mocks
+    public StandardManagedAuthorizer(AccessPolicyProvider accessPolicyProvider, UserGroupProvider userGroupProvider) {
+        this.accessPolicyProvider = accessPolicyProvider;
+        this.userGroupProvider = userGroupProvider;
+    }
+
+    @Override
+    public void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
+        accessPolicyProviderLookup = initializationContext.getAccessPolicyProviderLookup();
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        final PropertyValue accessPolicyProviderKey = configurationContext.getProperty("Access Policy Provider");
+        if (!accessPolicyProviderKey.isSet()) {
+            throw new AuthorizerCreationException("The Access Policy Provider must be set.");
+        }
+
+        accessPolicyProvider = accessPolicyProviderLookup.getAccessPolicyProvider(accessPolicyProviderKey.getValue());
+
+        // ensure the desired access policy provider was found
+        if (accessPolicyProvider == null) {
+            throw new AuthorizerCreationException(String.format("Unable to locate configured Access Policy Provider: %s", accessPolicyProviderKey));
+        }
+
+        userGroupProvider = accessPolicyProvider.getUserGroupProvider();
+
+        // ensure the desired access policy provider has a user group provider
+        if (userGroupProvider == null) {
+            throw new AuthorizerCreationException(String.format("Configured Access Policy Provider %s does not contain a User Group Provider", accessPolicyProviderKey));
+        }
+    }
+
+    @Override
+    public AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException {
+        final String resourceIdentifier = request.getResource().getIdentifier();
+        final AccessPolicy policy = accessPolicyProvider.getAccessPolicy(resourceIdentifier, request.getAction());
+        if (policy == null) {
+            return AuthorizationResult.resourceNotFound();
+        }
+
+        final UserAndGroups userAndGroups = userGroupProvider.getUserAndGroups(request.getIdentity());
+
+        final User user = userAndGroups.getUser();
+        if (user == null) {
+            return AuthorizationResult.denied(String.format("Unknown user with identity '%s'.", request.getIdentity()));
+        }
+
+        final Set<Group> userGroups = userAndGroups.getGroups();
+        if (policy.getUsers().contains(user.getIdentifier()) || containsGroup(userGroups, policy)) {
+            return AuthorizationResult.approved();
+        }
+
+        return AuthorizationResult.denied(request.getExplanationSupplier().get());
+    }
+
+    /**
+     * Determines if the policy contains one of the user's groups.
+     *
+     * @param userGroups the set of the user's groups
+     * @param policy the policy
+     * @return true if one of the Groups in userGroups is contained in the policy
+     */
+    private boolean containsGroup(final Set<Group> userGroups, final AccessPolicy policy) {
+        if (userGroups == null || userGroups.isEmpty() || policy.getGroups().isEmpty()) {
+            return false;
+        }
+
+        for (Group userGroup : userGroups) {
+            if (policy.getGroups().contains(userGroup.getIdentifier())) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    @Override
+    public String getFingerprint() throws AuthorizationAccessException {
+        XMLStreamWriter writer = null;
+        final StringWriter out = new StringWriter();
+        try {
+            writer = XML_OUTPUT_FACTORY.createXMLStreamWriter(out);
+            writer.writeStartDocument();
+            writer.writeStartElement("managedAuthorizations");
+
+            writer.writeStartElement(ACCESS_POLICY_PROVIDER_ELEMENT);
+            if (accessPolicyProvider instanceof ConfigurableAccessPolicyProvider) {
+                writer.writeCharacters(((ConfigurableAccessPolicyProvider) accessPolicyProvider).getFingerprint());
+            }
+            writer.writeEndElement();
+
+            writer.writeStartElement(USER_GROUP_PROVIDER_ELEMENT);
+            if (userGroupProvider instanceof ConfigurableUserGroupProvider) {
+                writer.writeCharacters(((ConfigurableUserGroupProvider) userGroupProvider).getFingerprint());
+            }
+            writer.writeEndElement();
+
+            writer.writeEndElement();
+            writer.writeEndDocument();
+            writer.flush();
+        } catch (XMLStreamException e) {
+            throw new AuthorizationAccessException("Unable to generate fingerprint", e);
+        } finally {
+            if (writer != null) {
+                try {
+                    writer.close();
+                } catch (XMLStreamException e) {
+                    // nothing to do here
+                }
+            }
+        }
+
+        return out.toString();
+    }
+
+    @Override
+    public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
+        if (StringUtils.isBlank(fingerprint)) {
+            return;
+        }
+
+        final FingerprintHolder fingerprintHolder = parseFingerprint(fingerprint);
+
+        if (StringUtils.isNotBlank(fingerprintHolder.getPolicyFingerprint()) && accessPolicyProvider instanceof ConfigurableAccessPolicyProvider) {
+            ((ConfigurableAccessPolicyProvider) accessPolicyProvider).inheritFingerprint(fingerprintHolder.getPolicyFingerprint());
+        }
+
+        if (StringUtils.isNotBlank(fingerprintHolder.getUserGroupFingerprint()) && userGroupProvider instanceof ConfigurableUserGroupProvider) {
+            ((ConfigurableUserGroupProvider) userGroupProvider).inheritFingerprint(fingerprintHolder.getUserGroupFingerprint());
+        }
+    }
+
+    @Override
+    public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
+        final FingerprintHolder fingerprintHolder = parseFingerprint(proposedFingerprint);
+
+        if (StringUtils.isNotBlank(fingerprintHolder.getPolicyFingerprint())) {
+            if (accessPolicyProvider instanceof ConfigurableAccessPolicyProvider) {
+                ((ConfigurableAccessPolicyProvider) accessPolicyProvider).checkInheritability(fingerprintHolder.getPolicyFingerprint());
+            } else {
+                throw new UninheritableAuthorizationsException("Policy fingerprint is not blank and the configured AccessPolicyProvider does not support fingerprinting.");
+            }
+        }
+
+        if (StringUtils.isNotBlank(fingerprintHolder.getUserGroupFingerprint())) {
+            if (userGroupProvider instanceof ConfigurableUserGroupProvider) {
+                ((ConfigurableUserGroupProvider) userGroupProvider).checkInheritability(fingerprintHolder.getUserGroupFingerprint());
+            } else {
+                throw new UninheritableAuthorizationsException("User/Group fingerprint is not blank and the configured UserGroupProvider does not support fingerprinting.");
+            }
+        }
+    }
+
+    private final FingerprintHolder parseFingerprint(final String fingerprint) throws AuthorizationAccessException {
+        final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
+
+        try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
+            final DocumentBuilder docBuilder = DOCUMENT_BUILDER_FACTORY.newDocumentBuilder();
+            final Document document = docBuilder.parse(in);
+            final Element rootElement = document.getDocumentElement();
+
+            final NodeList accessPolicyProviderList = rootElement.getElementsByTagName(ACCESS_POLICY_PROVIDER_ELEMENT);
+            if (accessPolicyProviderList.getLength() != 1) {
+                throw new AuthorizationAccessException(String.format("Only one %s element is allowed: %s", ACCESS_POLICY_PROVIDER_ELEMENT, fingerprint));
+            }
+
+            final NodeList userGroupProviderList = rootElement.getElementsByTagName(USER_GROUP_PROVIDER_ELEMENT);
+            if (userGroupProviderList.getLength() != 1) {
+                throw new AuthorizationAccessException(String.format("Only one %s element is allowed: %s", USER_GROUP_PROVIDER_ELEMENT, fingerprint));
+            }
+
+            final Node accessPolicyProvider = accessPolicyProviderList.item(0);
+            final Node userGroupProvider = userGroupProviderList.item(0);
+            return new FingerprintHolder(accessPolicyProvider.getTextContent(), userGroupProvider.getTextContent());
+        } catch (SAXException | ParserConfigurationException | IOException e) {
+            throw new AuthorizationAccessException("Unable to parse fingerprint", e);
+        }
+    }
+
+    @Override
+    public AccessPolicyProvider getAccessPolicyProvider() {
+        return accessPolicyProvider;
+    }
+
+    @Override
+    public void preDestruction() throws AuthorizerDestructionException {
+
+    }
+
+    private static class FingerprintHolder {
+        private final String policyFingerprint;
+        private final String userGroupFingerprint;
+
+        public FingerprintHolder(String policyFingerprint, String userGroupFingerprint) {
+            this.policyFingerprint = policyFingerprint;
+            this.userGroupFingerprint = userGroupFingerprint;
+        }
+
+        public String getPolicyFingerprint() {
+            return policyFingerprint;
+        }
+
+        public String getUserGroupFingerprint() {
+            return userGroupFingerprint;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/UsersAndAccessPolicies.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/UsersAndAccessPolicies.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/UsersAndAccessPolicies.java
new file mode 100644
index 0000000..7675f27
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/UsersAndAccessPolicies.java
@@ -0,0 +1,52 @@
+/*
+ * 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.authorization;
+
+import java.util.Set;
+
+/**
+ * A holder object to provide atomic access to policies for a given resource and users by
+ * identity. Implementations must ensure consistent access to the data backing this instance.
+ */
+public interface UsersAndAccessPolicies {
+
+    /**
+     * Retrieves the set of access policies for a given resource and action.
+     *
+     * @param resourceIdentifier the resource identifier to retrieve policies for
+     * @param action the action to retrieve policies for
+     * @return the access policy for the given resource and action
+     */
+    AccessPolicy getAccessPolicy(final String resourceIdentifier, final RequestAction action);
+
+    /**
+     * Retrieves a user by an identity string.
+     *
+     * @param identity the identity of the user to retrieve
+     * @return the user with the given identity
+     */
+    User getUser(final String identity);
+
+    /**
+     * Retrieves the groups for a given user identity.
+     *
+     * @param userIdentity a user identity
+     * @return the set of groups for the given user identity
+     */
+    Set<Group> getGroups(final String userIdentity);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/AuthorizationsHolder.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/AuthorizationsHolder.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/AuthorizationsHolder.java
new file mode 100644
index 0000000..6e84f49
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/AuthorizationsHolder.java
@@ -0,0 +1,187 @@
+/*
+ * 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.authorization.file;
+
+
+import org.apache.nifi.registry.security.authorization.file.generated.Authorizations;
+import org.apache.nifi.registry.security.authorization.file.generated.Policies;
+import org.apache.nifi.registry.security.authorization.AccessPolicy;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A holder to provide atomic access to data structures.
+ */
+public class AuthorizationsHolder {
+
+    private final Authorizations authorizations;
+
+    private final Set<AccessPolicy> allPolicies;
+    private final Map<String, Set<AccessPolicy>> policiesByResource;
+    private final Map<String, AccessPolicy> policiesById;
+
+    /**
+     * Creates a new holder and populates all convenience authorizations data structures.
+     *
+     * @param authorizations the current authorizations instance
+     */
+    public AuthorizationsHolder(final Authorizations authorizations) {
+        this.authorizations = authorizations;
+
+        // load all access policies
+        final Policies policies = authorizations.getPolicies();
+        final Set<AccessPolicy> allPolicies = Collections.unmodifiableSet(createAccessPolicies(policies));
+
+        // create a convenience map from resource id to policies
+        final Map<String, Set<AccessPolicy>> policiesByResourceMap = Collections.unmodifiableMap(createResourcePolicyMap(allPolicies));
+
+        // create a convenience map from policy id to policy
+        final Map<String, AccessPolicy> policiesByIdMap = Collections.unmodifiableMap(createPoliciesByIdMap(allPolicies));
+
+        // set all the holders
+        this.allPolicies = allPolicies;
+        this.policiesByResource = policiesByResourceMap;
+        this.policiesById = policiesByIdMap;
+    }
+
+    /**
+     * Creates AccessPolicies from the JAXB Policies.
+     *
+     * @param policies the JAXB Policies element
+     * @return a set of AccessPolicies corresponding to the provided Resources
+     */
+    private Set<AccessPolicy> createAccessPolicies(org.apache.nifi.registry.security.authorization.file.generated.Policies policies) {
+        Set<AccessPolicy> allPolicies = new HashSet<>();
+        if (policies == null || policies.getPolicy() == null) {
+            return allPolicies;
+        }
+
+        // load the new authorizations
+        for (final org.apache.nifi.registry.security.authorization.file.generated.Policy policy : policies.getPolicy()) {
+            final String policyIdentifier = policy.getIdentifier();
+            final String resourceIdentifier = policy.getResource();
+
+            // start a new builder and set the policy and resource identifiers
+            final AccessPolicy.Builder builder = new AccessPolicy.Builder()
+                    .identifier(policyIdentifier)
+                    .resource(resourceIdentifier);
+
+            // add each user identifier
+            for (org.apache.nifi.registry.security.authorization.file.generated.Policy.User user : policy.getUser()) {
+                builder.addUser(user.getIdentifier());
+            }
+
+            // add each group identifier
+            for (org.apache.nifi.registry.security.authorization.file.generated.Policy.Group group : policy.getGroup()) {
+                builder.addGroup(group.getIdentifier());
+            }
+
+            // add the appropriate request actions
+            final String authorizationCode = policy.getAction();
+            if (authorizationCode.equals(FileAccessPolicyProvider.READ_CODE)) {
+                builder.action(RequestAction.READ);
+            } else if (authorizationCode.equals(FileAccessPolicyProvider.WRITE_CODE)){
+                builder.action(RequestAction.WRITE);
+            } else if (authorizationCode.equals(FileAccessPolicyProvider.DELETE_CODE)){
+                builder.action(RequestAction.DELETE);
+            } else {
+                throw new IllegalStateException("Unknown Policy Action: " + authorizationCode);
+            }
+
+            // build the policy and add it to the map
+            allPolicies.add(builder.build());
+        }
+
+        return allPolicies;
+    }
+
+    /**
+     * Creates a map from resource identifier to the set of policies for the given resource.
+     *
+     * @param allPolicies the set of all policies
+     * @return a map from resource identifier to policies
+     */
+    private Map<String, Set<AccessPolicy>> createResourcePolicyMap(final Set<AccessPolicy> allPolicies) {
+        Map<String, Set<AccessPolicy>> resourcePolicies = new HashMap<>();
+
+        for (AccessPolicy policy : allPolicies) {
+            Set<AccessPolicy> policies = resourcePolicies.get(policy.getResource());
+            if (policies == null) {
+                policies = new HashSet<>();
+                resourcePolicies.put(policy.getResource(), policies);
+            }
+            policies.add(policy);
+        }
+
+        return resourcePolicies;
+    }
+
+    /**
+     * Creates a Map from policy identifier to AccessPolicy.
+     *
+     * @param policies the set of all access policies
+     * @return the Map from policy identifier to AccessPolicy
+     */
+    private Map<String, AccessPolicy> createPoliciesByIdMap(final Set<AccessPolicy> policies) {
+        Map<String,AccessPolicy> policyMap = new HashMap<>();
+        for (AccessPolicy policy : policies) {
+            policyMap.put(policy.getIdentifier(), policy);
+        }
+        return policyMap;
+    }
+
+    public Authorizations getAuthorizations() {
+        return authorizations;
+    }
+
+    public Set<AccessPolicy> getAllPolicies() {
+        return allPolicies;
+    }
+
+    public Map<String, Set<AccessPolicy>> getPoliciesByResource() {
+        return policiesByResource;
+    }
+
+    public Map<String, AccessPolicy> getPoliciesById() {
+        return policiesById;
+    }
+
+    public AccessPolicy getAccessPolicy(final String resourceIdentifier, final RequestAction action) {
+        if (resourceIdentifier == null) {
+            throw new IllegalArgumentException("Resource Identifier cannot be null");
+        }
+
+        final Set<AccessPolicy> resourcePolicies = policiesByResource.get(resourceIdentifier);
+        if (resourcePolicies == null) {
+            return null;
+        }
+
+        for (AccessPolicy accessPolicy : resourcePolicies) {
+            if (accessPolicy.getAction() == action) {
+                return accessPolicy;
+            }
+        }
+
+        return null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileAccessPolicyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileAccessPolicyProvider.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileAccessPolicyProvider.java
new file mode 100644
index 0000000..74887ad
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileAccessPolicyProvider.java
@@ -0,0 +1,757 @@
+/*
+ * 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.authorization.file;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.properties.util.IdentityMapping;
+import org.apache.nifi.registry.properties.util.IdentityMappingUtil;
+import org.apache.nifi.registry.security.authorization.AccessPolicy;
+import org.apache.nifi.registry.security.authorization.AccessPolicyProviderInitializationContext;
+import org.apache.nifi.registry.security.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.registry.security.authorization.ConfigurableAccessPolicyProvider;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.User;
+import org.apache.nifi.registry.security.authorization.UserGroupProvider;
+import org.apache.nifi.registry.security.authorization.UserGroupProviderLookup;
+import org.apache.nifi.registry.security.authorization.annotation.AuthorizerContext;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.registry.security.authorization.exception.UninheritableAuthorizationsException;
+import org.apache.nifi.registry.security.authorization.file.generated.Authorizations;
+import org.apache.nifi.registry.security.authorization.file.generated.Policies;
+import org.apache.nifi.registry.security.authorization.file.generated.Policy;
+import org.apache.nifi.registry.util.PropertyValue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamWriter;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Pattern;
+
+public class FileAccessPolicyProvider implements ConfigurableAccessPolicyProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(FileAccessPolicyProvider.class);
+
+    private static final String AUTHORIZATIONS_XSD = "/authorizations.xsd";
+    private static final String JAXB_AUTHORIZATIONS_PATH = "org.apache.nifi.registry.security.authorization.file.generated";
+
+    private static final JAXBContext JAXB_AUTHORIZATIONS_CONTEXT = initializeJaxbContext(JAXB_AUTHORIZATIONS_PATH);
+
+    /**
+     * Load the JAXBContext.
+     */
+    private static JAXBContext initializeJaxbContext(final String contextPath) {
+        try {
+            return JAXBContext.newInstance(contextPath, FileAuthorizer.class.getClassLoader());
+        } catch (JAXBException e) {
+            throw new RuntimeException("Unable to create JAXBContext.");
+        }
+    }
+
+    private static final DocumentBuilderFactory DOCUMENT_BUILDER_FACTORY = DocumentBuilderFactory.newInstance();
+    private static final XMLOutputFactory XML_OUTPUT_FACTORY = XMLOutputFactory.newInstance();
+
+    private static final String POLICY_ELEMENT = "policy";
+    private static final String POLICY_USER_ELEMENT = "policyUser";
+    private static final String POLICY_GROUP_ELEMENT = "policyGroup";
+    private static final String IDENTIFIER_ATTR = "identifier";
+    private static final String RESOURCE_ATTR = "resource";
+    private static final String ACTIONS_ATTR = "actions";
+
+    /* These codes must match the enumeration values set in authorizations.xsd */
+    static final String READ_CODE = "R";
+    static final String WRITE_CODE = "W";
+    static final String DELETE_CODE = "D";
+
+    /*  TODO - move this somewhere into nifi-registry-security-framework so it can be applied to any ConfigurableAccessPolicyProvider
+     *  (and also gets us away from requiring magic strings here) */
+    private static final ResourceActionPair[] INITIAL_ADMIN_ACCESS_POLICIES = {
+            new ResourceActionPair("/resources", READ_CODE),
+            new ResourceActionPair("/tenants", READ_CODE),
+            new ResourceActionPair("/tenants", WRITE_CODE),
+            new ResourceActionPair("/tenants", DELETE_CODE),
+            new ResourceActionPair("/policies", READ_CODE),
+            new ResourceActionPair("/policies", WRITE_CODE),
+            new ResourceActionPair("/policies", DELETE_CODE),
+            new ResourceActionPair("/buckets", READ_CODE),
+            new ResourceActionPair("/buckets", WRITE_CODE),
+            new ResourceActionPair("/buckets", DELETE_CODE),
+            new ResourceActionPair("/proxy", WRITE_CODE)
+    };
+
+    static final String PROP_NODE_IDENTITY_PREFIX = "Node Identity ";
+    static final String PROP_USER_GROUP_PROVIDER = "User Group Provider";
+    static final String PROP_AUTHORIZATIONS_FILE = "Authorizations File";
+    static final String PROP_INITIAL_ADMIN_IDENTITY = "Initial Admin Identity";
+    static final Pattern NODE_IDENTITY_PATTERN = Pattern.compile(PROP_NODE_IDENTITY_PREFIX + "\\S+");
+
+    private Schema authorizationsSchema;
+    private NiFiRegistryProperties properties;
+    private File authorizationsFile;
+    private String initialAdminIdentity;
+    private List<IdentityMapping> identityMappings;
+
+    private UserGroupProvider userGroupProvider;
+    private UserGroupProviderLookup userGroupProviderLookup;
+    private final AtomicReference<AuthorizationsHolder> authorizationsHolder = new AtomicReference<>();
+
+    @Override
+    public void initialize(AccessPolicyProviderInitializationContext initializationContext) throws AuthorizerCreationException {
+        userGroupProviderLookup = initializationContext.getUserGroupProviderLookup();
+
+        try {
+            final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
+            authorizationsSchema = schemaFactory.newSchema(FileAuthorizer.class.getResource(AUTHORIZATIONS_XSD));
+//            usersSchema = schemaFactory.newSchema(FileAuthorizer.class.getResource(USERS_XSD));
+        } catch (Exception e) {
+            throw new AuthorizerCreationException(e);
+        }
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        try {
+            final PropertyValue userGroupProviderIdentifier = configurationContext.getProperty(PROP_USER_GROUP_PROVIDER);
+            if (!userGroupProviderIdentifier.isSet()) {
+                throw new AuthorizerCreationException("The user group provider must be specified.");
+            }
+
+            userGroupProvider = userGroupProviderLookup.getUserGroupProvider(userGroupProviderIdentifier.getValue());
+            if (userGroupProvider == null) {
+                throw new AuthorizerCreationException("Unable to locate user group provider with identifier " + userGroupProviderIdentifier.getValue());
+            }
+
+            final PropertyValue authorizationsPath = configurationContext.getProperty(PROP_AUTHORIZATIONS_FILE);
+            if (StringUtils.isBlank(authorizationsPath.getValue())) {
+                throw new AuthorizerCreationException("The authorizations file must be specified.");
+            }
+
+            // get the authorizations file and ensure it exists
+            authorizationsFile = new File(authorizationsPath.getValue());
+            if (!authorizationsFile.exists()) {
+                logger.info("Creating new authorizations file at {}", new Object[] {authorizationsFile.getAbsolutePath()});
+                saveAuthorizations(new Authorizations());
+            }
+
+            // extract the identity mappings from nifi.properties if any are provided
+            identityMappings = Collections.unmodifiableList(IdentityMappingUtil.getIdentityMappings(properties));
+
+            // get the value of the initial admin identity
+            final PropertyValue initialAdminIdentityProp = configurationContext.getProperty(PROP_INITIAL_ADMIN_IDENTITY);
+            initialAdminIdentity = initialAdminIdentityProp.isSet() ? IdentityMappingUtil.mapIdentity(initialAdminIdentityProp.getValue(), identityMappings) : null;
+
+//            // extract any node identities
+//            nodeIdentities = new HashSet<>();
+//            for (Map.Entry<String,String> entry : configurationContext.getProperties().entrySet()) {
+//                Matcher matcher = NODE_IDENTITY_PATTERN.matcher(entry.getKey());
+//                if (matcher.matches() && !StringUtils.isBlank(entry.getValue())) {
+//                    nodeIdentities.add(IdentityMappingUtil.mapIdentity(entry.getValue(), identityMappings));
+//                }
+//            }
+
+            // load the authorizations
+            load();
+
+            logger.info(String.format("Authorizations file loaded at %s", new Date().toString()));
+        } catch (AuthorizerCreationException | JAXBException | IllegalStateException | SAXException e) {
+            throw new AuthorizerCreationException(e);
+        }
+    }
+
+    @Override
+    public UserGroupProvider getUserGroupProvider() {
+        return userGroupProvider;
+    }
+
+    @Override
+    public Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException {
+        return authorizationsHolder.get().getAllPolicies();
+    }
+
+    @Override
+    public synchronized AccessPolicy addAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
+        if (accessPolicy == null) {
+            throw new IllegalArgumentException("AccessPolicy cannot be null");
+        }
+
+        // create the new JAXB Policy
+        final Policy policy = createJAXBPolicy(accessPolicy);
+
+        // add the new Policy to the top-level list of policies
+        final AuthorizationsHolder holder = authorizationsHolder.get();
+        final Authorizations authorizations = holder.getAuthorizations();
+        authorizations.getPolicies().getPolicy().add(policy);
+
+        saveAndRefreshHolder(authorizations);
+
+        return authorizationsHolder.get().getPoliciesById().get(accessPolicy.getIdentifier());
+    }
+
+    @Override
+    public AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException {
+        if (identifier == null) {
+            return null;
+        }
+
+        final AuthorizationsHolder holder = authorizationsHolder.get();
+        return holder.getPoliciesById().get(identifier);
+    }
+
+    @Override
+    public AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) throws AuthorizationAccessException {
+        return authorizationsHolder.get().getAccessPolicy(resourceIdentifier, action);
+    }
+
+    @Override
+    public synchronized AccessPolicy updateAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
+        if (accessPolicy == null) {
+            throw new IllegalArgumentException("AccessPolicy cannot be null");
+        }
+
+        final AuthorizationsHolder holder = this.authorizationsHolder.get();
+        final Authorizations authorizations = holder.getAuthorizations();
+
+        // try to find an existing Authorization that matches the policy id
+        Policy updatePolicy = null;
+        for (Policy policy : authorizations.getPolicies().getPolicy()) {
+            if (policy.getIdentifier().equals(accessPolicy.getIdentifier())) {
+                updatePolicy = policy;
+                break;
+            }
+        }
+
+        // no matching Policy so return null
+        if (updatePolicy == null) {
+            return null;
+        }
+
+        // update the Policy, save, reload, and return
+        transferUsersAndGroups(accessPolicy, updatePolicy);
+        saveAndRefreshHolder(authorizations);
+
+        return this.authorizationsHolder.get().getPoliciesById().get(accessPolicy.getIdentifier());
+    }
+
+    @Override
+    public synchronized AccessPolicy deleteAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
+        if (accessPolicy == null) {
+            throw new IllegalArgumentException("AccessPolicy cannot be null");
+        }
+
+        return deleteAccessPolicy(accessPolicy.getIdentifier());
+    }
+
+    @Override
+    public synchronized AccessPolicy deleteAccessPolicy(String accessPolicyIdentifer) throws AuthorizationAccessException {
+        if (accessPolicyIdentifer == null) {
+            throw new IllegalArgumentException("Access policy identifier cannot be null");
+        }
+
+        final AuthorizationsHolder holder = this.authorizationsHolder.get();
+        AccessPolicy deletedPolicy = holder.getPoliciesById().get(accessPolicyIdentifer);
+        if (deletedPolicy == null) {
+            return null;
+        }
+
+        // find the matching Policy and remove it
+        final Authorizations authorizations = holder.getAuthorizations();
+        Iterator<Policy> policyIter = authorizations.getPolicies().getPolicy().iterator();
+        while (policyIter.hasNext()) {
+            final Policy policy = policyIter.next();
+            if (policy.getIdentifier().equals(accessPolicyIdentifer)) {
+                policyIter.remove();
+                break;
+            }
+        }
+
+        saveAndRefreshHolder(authorizations);
+        return deletedPolicy;
+    }
+
+    AuthorizationsHolder getAuthorizationsHolder() {
+        return authorizationsHolder.get();
+    }
+
+    @AuthorizerContext
+    public void setNiFiProperties(NiFiRegistryProperties properties) {
+        this.properties = properties;
+    }
+
+    @Override
+    public synchronized void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
+        parsePolicies(fingerprint).forEach(policy -> addAccessPolicy(policy));
+    }
+
+    @Override
+    public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
+        try {
+            // ensure we can understand the proposed fingerprint
+            parsePolicies(proposedFingerprint);
+        } catch (final AuthorizationAccessException e) {
+            throw new UninheritableAuthorizationsException("Unable to parse the proposed fingerprint: " + e);
+        }
+
+        // ensure we are in a proper state to inherit the fingerprint
+        if (!getAccessPolicies().isEmpty()) {
+            throw new UninheritableAuthorizationsException("Proposed fingerprint is not inheritable because the current access policies is not empty.");
+        }
+    }
+
+    @Override
+    public String getFingerprint() throws AuthorizationAccessException {
+        final List<AccessPolicy> policies = new ArrayList<>(getAccessPolicies());
+        Collections.sort(policies, Comparator.comparing(AccessPolicy::getIdentifier));
+
+        XMLStreamWriter writer = null;
+        final StringWriter out = new StringWriter();
+        try {
+            writer = XML_OUTPUT_FACTORY.createXMLStreamWriter(out);
+            writer.writeStartDocument();
+            writer.writeStartElement("accessPolicies");
+
+            for (AccessPolicy policy : policies) {
+                writePolicy(writer, policy);
+            }
+
+            writer.writeEndElement();
+            writer.writeEndDocument();
+            writer.flush();
+        } catch (XMLStreamException e) {
+            throw new AuthorizationAccessException("Unable to generate fingerprint", e);
+        } finally {
+            if (writer != null) {
+                try {
+                    writer.close();
+                } catch (XMLStreamException e) {
+                    // nothing to do here
+                }
+            }
+        }
+
+        return out.toString();
+    }
+
+    private List<AccessPolicy> parsePolicies(final String fingerprint) {
+        final List<AccessPolicy> policies = new ArrayList<>();
+
+        final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
+        try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
+            final DocumentBuilder docBuilder = DOCUMENT_BUILDER_FACTORY.newDocumentBuilder();
+            final Document document = docBuilder.parse(in);
+            final Element rootElement = document.getDocumentElement();
+
+            // parse all the policies and add them to the current access policy provider
+            NodeList policyNodes = rootElement.getElementsByTagName(POLICY_ELEMENT);
+            for (int i = 0; i < policyNodes.getLength(); i++) {
+                Node policyNode = policyNodes.item(i);
+                policies.add(parsePolicy((Element) policyNode));
+            }
+        } catch (SAXException | ParserConfigurationException | IOException e) {
+            throw new AuthorizationAccessException("Unable to parse fingerprint", e);
+        }
+
+        return policies;
+    }
+
+    private AccessPolicy parsePolicy(final Element element) {
+        final AccessPolicy.Builder builder = new AccessPolicy.Builder()
+                .identifier(element.getAttribute(IDENTIFIER_ATTR))
+                .resource(element.getAttribute(RESOURCE_ATTR));
+
+        final String actions = element.getAttribute(ACTIONS_ATTR);
+        if (actions.equals(RequestAction.READ.name())) {
+            builder.action(RequestAction.READ);
+        } else if (actions.equals(RequestAction.WRITE.name())) {
+            builder.action(RequestAction.WRITE);
+        } else if (actions.equals(RequestAction.DELETE.name())) {
+            builder.action(RequestAction.DELETE);
+        } else {
+            throw new IllegalStateException("Unknown Policy Action: " + actions);
+        }
+
+        NodeList policyUsers = element.getElementsByTagName(POLICY_USER_ELEMENT);
+        for (int i=0; i < policyUsers.getLength(); i++) {
+            Element policyUserNode = (Element) policyUsers.item(i);
+            builder.addUser(policyUserNode.getAttribute(IDENTIFIER_ATTR));
+        }
+
+        NodeList policyGroups = element.getElementsByTagName(POLICY_GROUP_ELEMENT);
+        for (int i=0; i < policyGroups.getLength(); i++) {
+            Element policyGroupNode = (Element) policyGroups.item(i);
+            builder.addGroup(policyGroupNode.getAttribute(IDENTIFIER_ATTR));
+        }
+
+        return builder.build();
+    }
+
+    private void writePolicy(final XMLStreamWriter writer, final AccessPolicy policy) throws XMLStreamException {
+        // sort the users for the policy
+        List<String> policyUsers = new ArrayList<>(policy.getUsers());
+        Collections.sort(policyUsers);
+
+        // sort the groups for this policy
+        List<String> policyGroups = new ArrayList<>(policy.getGroups());
+        Collections.sort(policyGroups);
+
+        writer.writeStartElement(POLICY_ELEMENT);
+        writer.writeAttribute(IDENTIFIER_ATTR, policy.getIdentifier());
+        writer.writeAttribute(RESOURCE_ATTR, policy.getResource());
+        writer.writeAttribute(ACTIONS_ATTR, policy.getAction().name());
+
+        for (String policyUser : policyUsers) {
+            writer.writeStartElement(POLICY_USER_ELEMENT);
+            writer.writeAttribute(IDENTIFIER_ATTR, policyUser);
+            writer.writeEndElement();
+        }
+
+        for (String policyGroup : policyGroups) {
+            writer.writeStartElement(POLICY_GROUP_ELEMENT);
+            writer.writeAttribute(IDENTIFIER_ATTR, policyGroup);
+            writer.writeEndElement();
+        }
+
+        writer.writeEndElement();
+    }
+
+    /**
+     * Loads the authorizations file and populates the AuthorizationsHolder, only called during start-up.
+     *
+     * @throws JAXBException            Unable to reload the authorized users file
+     */
+    private synchronized void load() throws JAXBException, SAXException {
+        // attempt to unmarshal
+        final Authorizations authorizations = unmarshallAuthorizations();
+        if (authorizations.getPolicies() == null) {
+            authorizations.setPolicies(new Policies());
+        }
+
+        final AuthorizationsHolder authorizationsHolder = new AuthorizationsHolder(authorizations);
+        final boolean emptyAuthorizations = authorizationsHolder.getAllPolicies().isEmpty();
+        final boolean hasInitialAdminIdentity = (initialAdminIdentity != null && !StringUtils.isBlank(initialAdminIdentity));
+
+        // if we are starting fresh then we might need to populate an initial admin
+        if (emptyAuthorizations && hasInitialAdminIdentity) {
+            logger.info("Populating authorizations for Initial Admin: " + initialAdminIdentity);
+            populateInitialAdmin(authorizations);
+            saveAndRefreshHolder(authorizations);
+        } else {
+            this.authorizationsHolder.set(authorizationsHolder);
+        }
+    }
+
+    private void saveAuthorizations(final Authorizations authorizations) throws JAXBException {
+        final Marshaller marshaller = JAXB_AUTHORIZATIONS_CONTEXT.createMarshaller();
+        marshaller.setSchema(authorizationsSchema);
+        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+        marshaller.marshal(authorizations, authorizationsFile);
+    }
+
+    private Authorizations unmarshallAuthorizations() throws JAXBException {
+        final Unmarshaller unmarshaller = JAXB_AUTHORIZATIONS_CONTEXT.createUnmarshaller();
+        unmarshaller.setSchema(authorizationsSchema);
+
+        final JAXBElement<Authorizations> element = unmarshaller.unmarshal(new StreamSource(authorizationsFile), Authorizations.class);
+        return element.getValue();
+    }
+
+    /**
+     *  Creates the initial admin user and sets policies managing buckets, users, and policies.
+     *
+     *  TODO - move this somewhere into nifi-registry-security-framework so it can be applied to any ConfigurableAccessPolicyProvider
+     */
+    private void populateInitialAdmin(final Authorizations authorizations) {
+        final User initialAdmin = userGroupProvider.getUserByIdentity(initialAdminIdentity);
+        if (initialAdmin == null) {
+            throw new AuthorizerCreationException("Unable to locate initial admin " + initialAdminIdentity + " to seed policies");
+        }
+
+        for (ResourceActionPair resourceAction : INITIAL_ADMIN_ACCESS_POLICIES) {
+            addUserToAccessPolicy(authorizations, resourceAction.resource, initialAdmin.getIdentifier(), resourceAction.actionCode);
+        }
+    }
+
+//    /**
+//     * Creates a user for each node and gives the nodes write permission to /proxy.
+//     *
+//     * @param authorizations the overall authorizations
+//     */
+//    private void populateNodes(Authorizations authorizations) {
+//        for (String nodeIdentity : nodeIdentities) {
+//            final User node = userGroupProvider.getUserByIdentity(nodeIdentity);
+//            if (node == null) {
+//                throw new AuthorizerCreationException("Unable to locate node " + nodeIdentity + " to seed policies.");
+//            }
+//
+//            // grant access to the proxy resource
+//            addUserToAccessPolicy(authorizations, ResourceType.Proxy.getValue(), node.getIdentifier(), WRITE_CODE);
+//
+//            // grant the user read/write access data of the root group
+//            if (rootGroupId != null) {
+//                addUserToAccessPolicy(authorizations, ResourceType.Data.getValue() + ResourceType.ProcessGroup.getValue() + "/" + rootGroupId, node.getIdentifier(), READ_CODE);
+//                addUserToAccessPolicy(authorizations, ResourceType.Data.getValue() + ResourceType.ProcessGroup.getValue() + "/" + rootGroupId, node.getIdentifier(), WRITE_CODE);
+//            }
+//        }
+//    }
+
+
+    /**
+     * Creates and adds an access policy for the given resource, identity, and actions to the specified authorizations.
+     *
+     * @param authorizations the Authorizations instance to add the policy to
+     * @param resource the resource for the policy
+     * @param userIdentifier the identifier for the user to add to the policy
+     * @param action the action for the policy
+     */
+    private void addUserToAccessPolicy(final Authorizations authorizations, final String resource, final String userIdentifier, final String action) {
+        // first try to find an existing policy for the given resource and action
+        Policy foundPolicy = null;
+        for (Policy policy : authorizations.getPolicies().getPolicy()) {
+            if (policy.getResource().equals(resource) && policy.getAction().equals(action)) {
+                foundPolicy = policy;
+                break;
+            }
+        }
+
+        if (foundPolicy == null) {
+            // if we didn't find an existing policy create a new one
+            final String uuidSeed = resource + action;
+
+            final AccessPolicy.Builder builder = new AccessPolicy.Builder()
+                    .identifierGenerateFromSeed(uuidSeed)
+                    .resource(resource)
+                    .addUser(userIdentifier);
+
+            if (action.equals(READ_CODE)) {
+                builder.action(RequestAction.READ);
+            } else if (action.equals(WRITE_CODE)) {
+                builder.action(RequestAction.WRITE);
+            } else if (action.equals(DELETE_CODE)) {
+                builder.action(RequestAction.DELETE);
+            } else {
+                throw new IllegalStateException("Unknown Policy Action: " + action);
+            }
+
+            final AccessPolicy accessPolicy = builder.build();
+            final Policy jaxbPolicy = createJAXBPolicy(accessPolicy);
+            authorizations.getPolicies().getPolicy().add(jaxbPolicy);
+        } else {
+            // otherwise add the user to the existing policy
+            Policy.User policyUser = new Policy.User();
+            policyUser.setIdentifier(userIdentifier);
+            foundPolicy.getUser().add(policyUser);
+        }
+    }
+
+    private Policy createJAXBPolicy(final AccessPolicy accessPolicy) {
+        final Policy policy = new Policy();
+        policy.setIdentifier(accessPolicy.getIdentifier());
+        policy.setResource(accessPolicy.getResource());
+
+        switch (accessPolicy.getAction()) {
+            case READ:
+                policy.setAction(READ_CODE);
+                break;
+            case WRITE:
+                policy.setAction(WRITE_CODE);
+                break;
+            case DELETE:
+                policy.setAction(DELETE_CODE);
+                break;
+            default:
+                break;
+        }
+
+        transferUsersAndGroups(accessPolicy, policy);
+        return policy;
+    }
+
+    /**
+     * Sets the given Policy to the state of the provided AccessPolicy. Users and Groups will be cleared and
+     * set to match the AccessPolicy, the resource and action will be set to match the AccessPolicy.
+     *
+     * Does not set the identifier.
+     *
+     * @param accessPolicy the AccessPolicy to transfer state from
+     * @param policy the Policy to transfer state to
+     */
+    private void transferUsersAndGroups(AccessPolicy accessPolicy, Policy policy) {
+        // add users to the policy
+        policy.getUser().clear();
+        for (String userIdentifier : accessPolicy.getUsers()) {
+            Policy.User policyUser = new Policy.User();
+            policyUser.setIdentifier(userIdentifier);
+            policy.getUser().add(policyUser);
+        }
+
+        // add groups to the policy
+        policy.getGroup().clear();
+        for (String groupIdentifier : accessPolicy.getGroups()) {
+            Policy.Group policyGroup = new Policy.Group();
+            policyGroup.setIdentifier(groupIdentifier);
+            policy.getGroup().add(policyGroup);
+        }
+    }
+
+    /**
+     * Adds the given user identifier to the policy if it doesn't already exist.
+     *
+     * @param userIdentifier a user identifier
+     * @param policy a policy to add the user to
+     */
+    private void addUserToPolicy(final String userIdentifier, final Policy policy) {
+        // determine if the user already exists in the policy
+        boolean userExists = false;
+        for (Policy.User policyUser : policy.getUser()) {
+            if (policyUser.getIdentifier().equals(userIdentifier)) {
+                userExists = true;
+                break;
+            }
+        }
+
+        // add the user to the policy if doesn't already exist
+        if (!userExists) {
+            Policy.User policyUser = new Policy.User();
+            policyUser.setIdentifier(userIdentifier);
+            policy.getUser().add(policyUser);
+        }
+    }
+
+    /**
+     * Adds the given group identifier to the policy if it doesn't already exist.
+     *
+     * @param groupIdentifier a group identifier
+     * @param policy a policy to add the user to
+     */
+    private void addGroupToPolicy(final String groupIdentifier, final Policy policy) {
+        // determine if the group already exists in the policy
+        boolean groupExists = false;
+        for (Policy.Group policyGroup : policy.getGroup()) {
+            if (policyGroup.getIdentifier().equals(groupIdentifier)) {
+                groupExists = true;
+                break;
+            }
+        }
+
+        // add the group to the policy if doesn't already exist
+        if (!groupExists) {
+            Policy.Group policyGroup = new Policy.Group();
+            policyGroup.setIdentifier(groupIdentifier);
+            policy.getGroup().add(policyGroup);
+        }
+    }
+
+    /**
+     * Finds the Policy matching the resource and action, or creates a new one and adds it to the list of policies.
+     *
+     * @param policies the policies to search through
+     * @param seedIdentity the seedIdentity to use when creating identifiers for new policies
+     * @param resource the resource for the policy
+     * @param action the action string for the police (R or RW)
+     * @return the matching policy or a new policy
+     */
+    private Policy getOrCreatePolicy(final List<Policy> policies, final String seedIdentity, final String resource, final String action) {
+        Policy foundPolicy = null;
+
+        // try to find a policy with the same resource and actions
+        for (Policy policy : policies) {
+            if (policy.getResource().equals(resource) && policy.getAction().equals(action)) {
+                foundPolicy = policy;
+                break;
+            }
+        }
+
+        // if a matching policy wasn't found then create one
+        if (foundPolicy == null) {
+            final String uuidSeed = resource + action + seedIdentity;
+            final String policyIdentifier = IdentifierUtil.getIdentifier(uuidSeed);
+
+            foundPolicy = new Policy();
+            foundPolicy.setIdentifier(policyIdentifier);
+            foundPolicy.setResource(resource);
+            foundPolicy.setAction(action);
+
+            policies.add(foundPolicy);
+        }
+
+        return foundPolicy;
+    }
+
+    /**
+     * Saves the Authorizations instance by marshalling to a file, then re-populates the
+     * in-memory data structures and sets the new holder.
+     *
+     * Synchronized to ensure only one thread writes the file at a time.
+     *
+     * @param authorizations the authorizations to save and populate from
+     * @throws AuthorizationAccessException if an error occurs saving the authorizations
+     */
+    private synchronized void saveAndRefreshHolder(final Authorizations authorizations) throws AuthorizationAccessException {
+        try {
+            saveAuthorizations(authorizations);
+
+            this.authorizationsHolder.set(new AuthorizationsHolder(authorizations));
+        } catch (JAXBException e) {
+            throw new AuthorizationAccessException("Unable to save Authorizations", e);
+        }
+    }
+
+    @Override
+    public void preDestruction() throws AuthorizerDestructionException {
+    }
+
+    private static class ResourceActionPair {
+        public String resource;
+        public String actionCode;
+        public ResourceActionPair(String resource, String actionCode) {
+            this.resource = resource;
+            this.actionCode = actionCode;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileAuthorizer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileAuthorizer.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileAuthorizer.java
new file mode 100644
index 0000000..e7104e3
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileAuthorizer.java
@@ -0,0 +1,288 @@
+/*
+ * 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.authorization.file;
+
+import org.apache.nifi.registry.security.authorization.StandardAuthorizerConfigurationContext;
+import org.apache.nifi.registry.security.authorization.annotation.AuthorizerContext;
+import org.apache.nifi.registry.security.authorization.AbstractPolicyBasedAuthorizer;
+import org.apache.nifi.registry.security.authorization.AccessPolicy;
+import org.apache.nifi.registry.security.authorization.AccessPolicyProviderInitializationContext;
+import org.apache.nifi.registry.security.authorization.AccessPolicyProviderLookup;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.AuthorizerInitializationContext;
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.User;
+import org.apache.nifi.registry.security.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.registry.security.authorization.UserGroupProviderLookup;
+import org.apache.nifi.registry.security.authorization.UsersAndAccessPolicies;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.regex.Matcher;
+
+/**
+ * Provides authorizes requests to resources using policies persisted in a file.
+ */
+public class FileAuthorizer extends AbstractPolicyBasedAuthorizer {
+
+    private static final Logger logger = LoggerFactory.getLogger(FileAuthorizer.class);
+
+    private static final String FILE_USER_GROUP_PROVIDER_ID = "file-user-group-provider";
+    private static final String FILE_ACCESS_POLICY_PROVIDER_ID = "file-access-policy-provider";
+
+    static final String PROP_LEGACY_AUTHORIZED_USERS_FILE = "Legacy Authorized Users File";
+
+    private FileUserGroupProvider userGroupProvider = new FileUserGroupProvider();
+    private FileAccessPolicyProvider accessPolicyProvider = new FileAccessPolicyProvider();
+
+    @Override
+    public void initialize(final AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException {
+        // initialize the user group provider
+        userGroupProvider.initialize(new UserGroupProviderInitializationContext() {
+            @Override
+            public String getIdentifier() {
+                return FILE_USER_GROUP_PROVIDER_ID;
+            }
+
+            @Override
+            public UserGroupProviderLookup getUserGroupProviderLookup() {
+                return (identifier) -> null;
+            }
+        });
+
+        // initialize the access policy provider
+        accessPolicyProvider.initialize(new AccessPolicyProviderInitializationContext() {
+            @Override
+            public String getIdentifier() {
+                return FILE_ACCESS_POLICY_PROVIDER_ID;
+            }
+
+            @Override
+            public UserGroupProviderLookup getUserGroupProviderLookup() {
+                return (identifier) -> {
+                    if (FILE_USER_GROUP_PROVIDER_ID.equals(identifier)) {
+                        return userGroupProvider;
+                    }
+
+                    return null;
+                };
+            }
+
+            @Override
+            public AccessPolicyProviderLookup getAccessPolicyProviderLookup() {
+                return (identifier) ->  null;
+            }
+        });
+    }
+
+    @Override
+    public void doOnConfigured(final AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        final Map<String, String> configurationProperties = configurationContext.getProperties();
+
+        // relay the relevant config
+        final Map<String, String> userGroupProperties = new HashMap<>();
+        if (configurationProperties.containsKey(FileUserGroupProvider.PROP_TENANTS_FILE)) {
+            userGroupProperties.put(FileUserGroupProvider.PROP_TENANTS_FILE, configurationProperties.get(FileUserGroupProvider.PROP_TENANTS_FILE));
+        }
+        if (configurationProperties.containsKey(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)) {
+            userGroupProperties.put(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE, configurationProperties.get(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE));
+        }
+
+        // relay the relevant config
+        final Map<String, String> accessPolicyProperties = new HashMap<>();
+        accessPolicyProperties.put(FileAccessPolicyProvider.PROP_USER_GROUP_PROVIDER, FILE_USER_GROUP_PROVIDER_ID);
+        if (configurationProperties.containsKey(FileAccessPolicyProvider.PROP_AUTHORIZATIONS_FILE)) {
+            accessPolicyProperties.put(FileAccessPolicyProvider.PROP_AUTHORIZATIONS_FILE, configurationProperties.get(FileAccessPolicyProvider.PROP_AUTHORIZATIONS_FILE));
+        }
+        if (configurationProperties.containsKey(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)) {
+            accessPolicyProperties.put(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY, configurationProperties.get(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY));
+        }
+        if (configurationProperties.containsKey(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE)) {
+            accessPolicyProperties.put(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE, configurationProperties.get(FileAuthorizer.PROP_LEGACY_AUTHORIZED_USERS_FILE));
+        }
+
+        // ensure all node identities are seeded into the user provider
+        configurationProperties.forEach((property, value) -> {
+            final Matcher matcher = FileAccessPolicyProvider.NODE_IDENTITY_PATTERN.matcher(property);
+            if (matcher.matches()) {
+                accessPolicyProperties.put(property, value);
+                userGroupProperties.put(property.replace(FileAccessPolicyProvider.PROP_NODE_IDENTITY_PREFIX, FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX), value);
+            }
+        });
+
+        // ensure the initial admin is seeded into the user provider if appropriate
+        if (configurationProperties.containsKey(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY)) {
+            int i = 0;
+            while (true) {
+                final String key = FileUserGroupProvider.PROP_INITIAL_USER_IDENTITY_PREFIX + i++;
+                if (!userGroupProperties.containsKey(key)) {
+                    userGroupProperties.put(key, configurationProperties.get(FileAccessPolicyProvider.PROP_INITIAL_ADMIN_IDENTITY));
+                    break;
+                }
+            }
+        }
+
+        // configure the user group provider
+        userGroupProvider.onConfigured(new StandardAuthorizerConfigurationContext(FILE_USER_GROUP_PROVIDER_ID, userGroupProperties));
+
+        // configure the access policy provider
+        accessPolicyProvider.onConfigured(new StandardAuthorizerConfigurationContext(FILE_USER_GROUP_PROVIDER_ID, accessPolicyProperties));
+    }
+
+    @Override
+    public void preDestruction() {
+
+    }
+
+    // ------------------ Groups ------------------
+
+    @Override
+    public synchronized Group doAddGroup(Group group) throws AuthorizationAccessException {
+        return userGroupProvider.addGroup(group);
+    }
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        return userGroupProvider.getGroup(identifier);
+    }
+
+    @Override
+    public synchronized Group doUpdateGroup(Group group) throws AuthorizationAccessException {
+        return userGroupProvider.updateGroup(group);
+    }
+
+    @Override
+    public synchronized Group deleteGroup(Group group) throws AuthorizationAccessException {
+        return userGroupProvider.deleteGroup(group);
+    }
+
+    @Override
+    public synchronized Group deleteGroup(String groupId) throws AuthorizationAccessException {
+        return userGroupProvider.deleteGroup(groupId);
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return userGroupProvider.getGroups();
+    }
+
+    // ------------------ Users ------------------
+
+    @Override
+    public synchronized User doAddUser(final User user) throws AuthorizationAccessException {
+        return userGroupProvider.addUser(user);
+    }
+
+    @Override
+    public User getUser(final String identifier) throws AuthorizationAccessException {
+        return userGroupProvider.getUser(identifier);
+    }
+
+    @Override
+    public User getUserByIdentity(final String identity) throws AuthorizationAccessException {
+        return userGroupProvider.getUserByIdentity(identity);
+    }
+
+    @Override
+    public synchronized User doUpdateUser(final User user) throws AuthorizationAccessException {
+        return userGroupProvider.updateUser(user);
+    }
+
+    @Override
+    public synchronized User deleteUser(final User user) throws AuthorizationAccessException {
+        return userGroupProvider.deleteUser(user);
+    }
+
+    @Override
+    public synchronized User deleteUser(final String userId) throws AuthorizationAccessException {
+        return userGroupProvider.deleteUser(userId);
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return userGroupProvider.getUsers();
+    }
+
+    // ------------------ AccessPolicies ------------------
+
+    @Override
+    public synchronized AccessPolicy doAddAccessPolicy(final AccessPolicy accessPolicy) throws AuthorizationAccessException {
+        return accessPolicyProvider.addAccessPolicy(accessPolicy);
+    }
+
+    @Override
+    public AccessPolicy getAccessPolicy(final String identifier) throws AuthorizationAccessException {
+        return accessPolicyProvider.getAccessPolicy(identifier);
+    }
+
+    @Override
+    public synchronized AccessPolicy updateAccessPolicy(final AccessPolicy accessPolicy) throws AuthorizationAccessException {
+        return accessPolicyProvider.updateAccessPolicy(accessPolicy);
+    }
+
+    @Override
+    public synchronized AccessPolicy deleteAccessPolicy(final AccessPolicy accessPolicy) throws AuthorizationAccessException {
+        return accessPolicyProvider.deleteAccessPolicy(accessPolicy);
+    }
+
+    @Override
+    public synchronized AccessPolicy deleteAccessPolicy(final String accessPolicyIdentifier) throws AuthorizationAccessException {
+        return accessPolicyProvider.deleteAccessPolicy(accessPolicyIdentifier);
+    }
+
+    @Override
+    public Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException {
+        return accessPolicyProvider.getAccessPolicies();
+    }
+
+    @AuthorizerContext
+    public void setNiFiProperties(NiFiRegistryProperties properties) {
+        userGroupProvider.setNiFiProperties(properties);
+        accessPolicyProvider.setNiFiProperties(properties);
+    }
+
+    @Override
+    public synchronized UsersAndAccessPolicies getUsersAndAccessPolicies() throws AuthorizationAccessException {
+        final AuthorizationsHolder authorizationsHolder = accessPolicyProvider.getAuthorizationsHolder();
+        final UserGroupHolder userGroupHolder = userGroupProvider.getUserGroupHolder();
+
+        return new UsersAndAccessPolicies() {
+            @Override
+            public AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) {
+                return authorizationsHolder.getAccessPolicy(resourceIdentifier, action);
+            }
+
+            @Override
+            public User getUser(String identity) {
+                return userGroupHolder.getUser(identity);
+            }
+
+            @Override
+            public Set<Group> getGroups(String userIdentity) {
+                return userGroupHolder.getGroups(userIdentity);
+            }
+        };
+    }
+
+}


[15/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/ResourceType.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/ResourceType.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/ResourceType.java
deleted file mode 100644
index 4987a38..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/resource/ResourceType.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.resource;
-
-public enum ResourceType {
-    Bucket("/buckets"),
-    Policy("/policies"),
-    Proxy("/proxy"),
-    Resource("/resources"),
-    Tenant("/tenants");
-
-    final String value;
-
-    private ResourceType(final String value) {
-        this.value = value;
-    }
-
-    public String getValue() {
-        return value;
-    }
-
-    public static ResourceType valueOfValue(final String rawValue) {
-        ResourceType type = null;
-
-        for (final ResourceType rt : values()) {
-            if (rt.getValue().equals(rawValue)) {
-                type = rt;
-                break;
-            }
-        }
-
-        if (type == null) {
-            throw new IllegalArgumentException("Unknown resource type value " + rawValue);
-        }
-
-        return type;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUser.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUser.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUser.java
deleted file mode 100644
index 5f49241..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUser.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.nifi.registry.authorization.user;
-
-import java.util.Set;
-
-/**
- * A representation of a NiFi user that has logged into the application
- */
-public interface NiFiUser {
-
-    /**
-     * @return the unique identity of this user
-     */
-    String getIdentity();
-
-    /**
-     * @return the groups that this user belongs to if this nifi is configured to load user groups, null otherwise.
-     */
-    Set<String> getGroups();
-
-    /**
-     * @return the next user in the proxied entities chain, or <code>null</code> if no more users exist in the chain.
-     */
-    NiFiUser getChain();
-
-    /**
-     * @return <code>true</code> if the user is the unauthenticated Anonymous user
-     */
-    boolean isAnonymous();
-
-    /**
-     * @return the address of the client that made the request which created this user
-     */
-    String getClientAddress();
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUserDetails.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUserDetails.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUserDetails.java
deleted file mode 100644
index 7b0da47..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUserDetails.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.user;
-
-import org.apache.commons.lang3.StringUtils;
-import org.springframework.security.core.GrantedAuthority;
-import org.springframework.security.core.userdetails.UserDetails;
-
-import java.util.Collection;
-import java.util.Collections;
-
-/**
- * User details for a NiFi user.
- */
-public class NiFiUserDetails implements UserDetails {
-
-    private final NiFiUser user;
-
-    /**
-     * Creates a new NiFiUserDetails.
-     *
-     * @param user user
-     */
-    public NiFiUserDetails(NiFiUser user) {
-        this.user = user;
-    }
-
-    /**
-     * Get the user for this UserDetails.
-     *
-     * @return user
-     */
-    public NiFiUser getNiFiUser() {
-        return user;
-    }
-
-    /**
-     * Returns the authorities that this NiFi user has.
-     *
-     * @return authorities
-     */
-    @Override
-    public Collection<? extends GrantedAuthority> getAuthorities() {
-        return Collections.EMPTY_SET;
-    }
-
-    @Override
-    public String getPassword() {
-        return StringUtils.EMPTY;
-    }
-
-    @Override
-    public String getUsername() {
-        return user.getIdentity();
-    }
-
-    @Override
-    public boolean isAccountNonExpired() {
-        return true;
-    }
-
-    @Override
-    public boolean isAccountNonLocked() {
-        return true;
-    }
-
-    @Override
-    public boolean isCredentialsNonExpired() {
-        return true;
-    }
-
-    @Override
-    public boolean isEnabled() {
-        return true;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUserUtils.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUserUtils.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUserUtils.java
deleted file mode 100644
index 2ad508a..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/NiFiUserUtils.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.user;
-
-import org.apache.commons.lang3.StringUtils;
-import org.springframework.security.core.Authentication;
-import org.springframework.security.core.context.SecurityContext;
-import org.springframework.security.core.context.SecurityContextHolder;
-
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Utility methods for retrieving information about the current application user.
- *
- */
-public final class NiFiUserUtils {
-
-    /**
-     * Returns the current NiFiUser or null if the current user is not a NiFiUser.
-     *
-     * @return user
-     */
-    public static NiFiUser getNiFiUser() {
-        NiFiUser user = null;
-
-        // obtain the principal in the current authentication
-        final SecurityContext context = SecurityContextHolder.getContext();
-        final Authentication authentication = context.getAuthentication();
-        if (authentication != null) {
-            Object principal = authentication.getPrincipal();
-            if (principal instanceof NiFiUserDetails) {
-                user = ((NiFiUserDetails) principal).getNiFiUser();
-            }
-        }
-
-        return user;
-    }
-
-    public static String getNiFiUserIdentity() {
-        // get the nifi user to extract the username
-        NiFiUser user = NiFiUserUtils.getNiFiUser();
-        if (user == null) {
-            return "unknown";
-        } else {
-            return user.getIdentity();
-        }
-    }
-
-    /**
-     * Builds the proxy chain for the specified user.
-     *
-     * @param user The current user
-     * @return The proxy chain for that user in List form
-     */
-    public static List<String> buildProxiedEntitiesChain(final NiFiUser user) {
-        // calculate the dn chain
-        final List<String> proxyChain = new ArrayList<>();
-
-        // build the dn chain
-        NiFiUser chainedUser = user;
-        while (chainedUser != null) {
-            // add the entry for this user
-            if (chainedUser.isAnonymous()) {
-                // use an empty string to represent an anonymous user in the proxy entities chain
-                proxyChain.add(StringUtils.EMPTY);
-            } else {
-                proxyChain.add(chainedUser.getIdentity());
-            }
-
-            // go to the next user in the chain
-            chainedUser = chainedUser.getChain();
-        }
-
-        return proxyChain;
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/StandardNiFiUser.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/StandardNiFiUser.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/StandardNiFiUser.java
deleted file mode 100644
index 3a8a8bc..0000000
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/authorization/user/StandardNiFiUser.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.user;
-
-import org.apache.commons.lang3.StringUtils;
-
-import java.util.Collections;
-import java.util.Objects;
-import java.util.Set;
-
-/**
- * An implementation of NiFiUser.
- */
-public class StandardNiFiUser implements NiFiUser {
-
-    public static final String ANONYMOUS_IDENTITY = "anonymous";
-    public static final StandardNiFiUser ANONYMOUS = new Builder().identity(ANONYMOUS_IDENTITY).anonymous(true).build();
-
-    private final String identity;
-    private final Set<String> groups;
-    private final NiFiUser chain;
-    private final String clientAddress;
-    private final boolean isAnonymous;
-
-    private StandardNiFiUser(final Builder builder) {
-        this.identity = builder.identity;
-        this.groups = builder.groups == null ? null : Collections.unmodifiableSet(builder.groups);
-        this.chain = builder.chain;
-        this.clientAddress = builder.clientAddress;
-        this.isAnonymous = builder.isAnonymous;
-    }
-
-    /**
-     * This static builder allows the chain and clientAddress to be populated without allowing calling code to provide a non-anonymous identity of the anonymous user.
-     *
-     * @param chain the proxied entities in {@see NiFiUser} form
-     * @param clientAddress the address the request originated from
-     * @return an anonymous user instance with the identity "anonymous"
-     */
-    public static StandardNiFiUser populateAnonymousUser(NiFiUser chain, String clientAddress) {
-        return new Builder().identity(ANONYMOUS_IDENTITY).chain(chain).clientAddress(clientAddress).anonymous(true).build();
-    }
-
-    @Override
-    public String getIdentity() {
-        return identity;
-    }
-
-    @Override
-    public Set<String> getGroups() {
-        return groups;
-    }
-
-    @Override
-    public NiFiUser getChain() {
-        return chain;
-    }
-
-    @Override
-    public boolean isAnonymous() {
-        return isAnonymous;
-    }
-
-    @Override
-    public String getClientAddress() {
-        return clientAddress;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-
-        if (!(obj instanceof NiFiUser)) {
-            return false;
-        }
-
-        final NiFiUser other = (NiFiUser) obj;
-        return Objects.equals(this.identity, other.getIdentity());
-    }
-
-    @Override
-    public int hashCode() {
-        int hash = 7;
-        hash = 53 * hash + Objects.hashCode(this.identity);
-        return hash;
-    }
-
-    @Override
-    public String toString() {
-        final String formattedGroups;
-        if (groups == null) {
-            formattedGroups = "none";
-        } else {
-            formattedGroups = StringUtils.join(groups, ", ");
-        }
-
-        return String.format("identity[%s], groups[%s]", getIdentity(), formattedGroups);
-    }
-
-    /**
-     * Builder for a StandardNiFiUser
-     */
-    public static class Builder {
-
-        private String identity;
-        private Set<String> groups;
-        private NiFiUser chain;
-        private String clientAddress;
-        private boolean isAnonymous = false;
-
-        /**
-         * Sets the identity.
-         *
-         * @param identity the identity string for the user (i.e. "Andy" or "CN=alopresto, OU=Apache NiFi")
-         * @return the builder
-         */
-        public Builder identity(final String identity) {
-            this.identity = identity;
-            return this;
-        }
-
-        /**
-         * Sets the groups.
-         *
-         * @param groups the user groups
-         * @return the builder
-         */
-        public Builder groups(final Set<String> groups) {
-            this.groups = groups;
-            return this;
-        }
-
-        /**
-         * Sets the chain.
-         *
-         * @param chain the proxy chain that leads to this users
-         * @return the builder
-         */
-        public Builder chain(final NiFiUser chain) {
-            this.chain = chain;
-            return this;
-        }
-
-        /**
-         * Sets the client address.
-         *
-         * @param clientAddress the source address of the request
-         * @return the builder
-         */
-        public Builder clientAddress(final String clientAddress) {
-            this.clientAddress = clientAddress;
-            return this;
-        }
-
-        /**
-         * Sets whether this user is the canonical "anonymous" user
-         *
-         * @param isAnonymous true to represent the canonical "anonymous" user
-         * @return the builder
-         */
-        private Builder anonymous(final boolean isAnonymous) {
-            this.isAnonymous = isAnonymous;
-            return this;
-        }
-
-        /**
-         * @return builds a StandardNiFiUser from the current state of the builder
-         */
-        public StandardNiFiUser build() {
-            return new StandardNiFiUser(this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseKeyService.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseKeyService.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseKeyService.java
new file mode 100644
index 0000000..eb55251
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/DatabaseKeyService.java
@@ -0,0 +1,117 @@
+/*
+ * 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.db;
+
+import org.apache.nifi.registry.db.entity.KeyEntity;
+import org.apache.nifi.registry.db.repository.KeyRepository;
+import org.apache.nifi.registry.security.key.Key;
+import org.apache.nifi.registry.security.key.KeyService;
+import org.apache.nifi.registry.service.DataModelMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+import java.util.UUID;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+@Service
+public class DatabaseKeyService implements KeyService {
+
+    private static final Logger logger = LoggerFactory.getLogger(DatabaseKeyService.class);
+
+    private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
+    private final Lock readLock = lock.readLock();
+    private final Lock writeLock = lock.writeLock();
+
+    private KeyRepository keyRepository;
+
+    @Autowired
+    public DatabaseKeyService(KeyRepository keyRepository) {
+        this.keyRepository = keyRepository;
+    }
+
+    @Override
+    public Key getKey(String id) {
+        if (id == null) {
+            throw new IllegalArgumentException("Id cannot be null");
+        }
+
+        Key key = null;
+        readLock.lock();
+        try {
+            KeyEntity keyEntity = keyRepository.findOne(id);
+            if (keyEntity != null) {
+                key = DataModelMapper.map(keyEntity);
+            } else {
+                logger.debug("No signing key found with id='" + id + "'");
+            }
+        } finally {
+            readLock.unlock();
+        }
+        return key;
+    }
+
+    @Override
+    public Key getOrCreateKey(String identity) {
+        if (identity == null) {
+            throw new IllegalArgumentException("Identity cannot be null");
+        }
+
+        Key key;
+        writeLock.lock();
+        try {
+            final KeyEntity existingKeyEntity = keyRepository.findOneByTenantIdentity(identity);
+            if (existingKeyEntity == null) {
+                logger.debug("No key found with identity='" + identity + "'. Creating new key.");
+
+                final KeyEntity newKeyEntity = new KeyEntity();
+                newKeyEntity.setId(UUID.randomUUID().toString());
+                newKeyEntity.setTenantIdentity(identity);
+                newKeyEntity.setKeyValue(UUID.randomUUID().toString());
+
+                final KeyEntity savedKeyEntity = keyRepository.save(newKeyEntity);
+
+                key = DataModelMapper.map(savedKeyEntity);
+            } else {
+                key = DataModelMapper.map(existingKeyEntity);
+            }
+        } finally {
+            writeLock.unlock();
+        }
+        return key;
+    }
+
+    @Override
+    public void deleteKey(String identity) {
+        if (identity == null) {
+            throw new IllegalArgumentException("Identity cannot be null");
+        }
+
+        Key key;
+        writeLock.lock();
+        try {
+            logger.debug("Deleting key with identity='" + identity + "'.");
+            keyRepository.deleteByTenantIdentity(identity);
+        } finally {
+            writeLock.unlock();
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/KeyEntity.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/KeyEntity.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/KeyEntity.java
new file mode 100644
index 0000000..7ec7d22
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/entity/KeyEntity.java
@@ -0,0 +1,61 @@
+/*
+ * 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.db.entity;
+
+import javax.persistence.Column;
+import javax.persistence.Entity;
+import javax.persistence.Id;
+import javax.persistence.Table;
+
+@Entity
+@Table(name = "SIGNING_KEY")
+public class KeyEntity {
+
+    @Id
+    private String id;
+
+    @Column(name = "TENANT_IDENTITY", unique = true, nullable = false)
+    private String tenantIdentity;
+
+    @Column(name = "KEY_VALUE", nullable = false)
+    private String keyValue;
+
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
+
+    public String getTenantIdentity() {
+        return tenantIdentity;
+    }
+
+    public void setTenantIdentity(String tenantIdentity) {
+        this.tenantIdentity = tenantIdentity;
+    }
+
+    public String getKeyValue() {
+        return keyValue;
+    }
+
+    public void setKeyValue(String keyValue) {
+        this.keyValue = keyValue;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/KeyRepository.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/KeyRepository.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/KeyRepository.java
new file mode 100644
index 0000000..1f31776
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/db/repository/KeyRepository.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.db.repository;
+
+import org.apache.nifi.registry.db.entity.KeyEntity;
+import org.springframework.data.repository.CrudRepository;
+
+/**
+ * Spring Data Repository for KeyEntity.
+ */
+public interface KeyRepository extends CrudRepository<KeyEntity, String> {
+
+    KeyEntity findOneByTenantIdentity(String identity);
+
+    void deleteByTenantIdentity(String identity);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/extension/ExtensionManager.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/extension/ExtensionManager.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/extension/ExtensionManager.java
index d3703a4..27e8b91 100644
--- a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/extension/ExtensionManager.java
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/extension/ExtensionManager.java
@@ -17,9 +17,10 @@
 package org.apache.nifi.registry.extension;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.authorization.AccessPolicyProvider;
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.UserGroupProvider;
+import org.apache.nifi.registry.security.authentication.LoginIdentityProvider;
+import org.apache.nifi.registry.security.authorization.AccessPolicyProvider;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.UserGroupProvider;
 import org.apache.nifi.registry.flow.FlowPersistenceProvider;
 import org.apache.nifi.registry.properties.NiFiRegistryProperties;
 import org.slf4j.Logger;
@@ -54,6 +55,7 @@ public class ExtensionManager {
         classes.add(UserGroupProvider.class);
         classes.add(AccessPolicyProvider.class);
         classes.add(Authorizer.class);
+        classes.add(LoginIdentityProvider.class);
         EXTENSION_CLASSES = Collections.unmodifiableList(classes);
     }
 

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AbstractPolicyBasedAuthorizer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AbstractPolicyBasedAuthorizer.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AbstractPolicyBasedAuthorizer.java
new file mode 100644
index 0000000..734a983
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AbstractPolicyBasedAuthorizer.java
@@ -0,0 +1,839 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.registry.security.authorization.AccessPolicy;
+import org.apache.nifi.registry.security.authorization.AccessPolicyProvider;
+import org.apache.nifi.registry.security.authorization.AccessPolicyProviderInitializationContext;
+import org.apache.nifi.registry.security.authorization.AuthorizationRequest;
+import org.apache.nifi.registry.security.authorization.AuthorizationResult;
+import org.apache.nifi.registry.security.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.registry.security.authorization.ConfigurableAccessPolicyProvider;
+import org.apache.nifi.registry.security.authorization.ConfigurableUserGroupProvider;
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.ManagedAuthorizer;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.User;
+import org.apache.nifi.registry.security.authorization.UserAndGroups;
+import org.apache.nifi.registry.security.authorization.UserGroupProvider;
+import org.apache.nifi.registry.security.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.registry.security.authorization.exception.UninheritableAuthorizationsException;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamWriter;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Set;
+
+/**
+ * An Authorizer that provides management of users, groups, and policies.
+ */
+public abstract class AbstractPolicyBasedAuthorizer implements ManagedAuthorizer {
+
+    static final DocumentBuilderFactory DOCUMENT_BUILDER_FACTORY = DocumentBuilderFactory.newInstance();
+    static final XMLOutputFactory XML_OUTPUT_FACTORY = XMLOutputFactory.newInstance();
+
+    static final String USER_ELEMENT = "user";
+    static final String GROUP_USER_ELEMENT = "groupUser";
+    static final String GROUP_ELEMENT = "group";
+    static final String POLICY_ELEMENT = "policy";
+    static final String POLICY_USER_ELEMENT = "policyUser";
+    static final String POLICY_GROUP_ELEMENT = "policyGroup";
+    static final String IDENTIFIER_ATTR = "identifier";
+    static final String IDENTITY_ATTR = "identity";
+    static final String NAME_ATTR = "name";
+    static final String RESOURCE_ATTR = "resource";
+    static final String ACTIONS_ATTR = "actions";
+
+    @Override
+    public final void onConfigured(final AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        doOnConfigured(configurationContext);
+    }
+
+    /**
+     * Allows sub-classes to take action when onConfigured is called.
+     *
+     * @param configurationContext the configuration context
+     * @throws AuthorizerCreationException if an error occurs during onConfigured process
+     */
+    protected abstract void doOnConfigured(final AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException;
+
+    @Override
+    public final AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException {
+        final UsersAndAccessPolicies usersAndAccessPolicies = getUsersAndAccessPolicies();
+        final String resourceIdentifier = request.getResource().getIdentifier();
+
+        final AccessPolicy policy = usersAndAccessPolicies.getAccessPolicy(resourceIdentifier, request.getAction());
+        if (policy == null) {
+            return AuthorizationResult.resourceNotFound();
+        }
+
+        final User user = usersAndAccessPolicies.getUser(request.getIdentity());
+        if (user == null) {
+            return AuthorizationResult.denied(String.format("Unknown user with identity '%s'.", request.getIdentity()));
+        }
+
+        final Set<Group> userGroups = usersAndAccessPolicies.getGroups(user.getIdentity());
+        if (policy.getUsers().contains(user.getIdentifier()) || containsGroup(userGroups, policy)) {
+            return AuthorizationResult.approved();
+        }
+
+        return AuthorizationResult.denied(request.getExplanationSupplier().get());
+    }
+
+    /**
+     * Determines if the policy contains one of the user's groups.
+     *
+     * @param userGroups the set of the user's groups
+     * @param policy the policy
+     * @return true if one of the Groups in userGroups is contained in the policy
+     */
+    private boolean containsGroup(final Set<Group> userGroups, final AccessPolicy policy) {
+        if (userGroups.isEmpty() || policy.getGroups().isEmpty()) {
+            return false;
+        }
+
+        for (Group userGroup : userGroups) {
+            if (policy.getGroups().contains(userGroup.getIdentifier())) {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    /**
+     * Adds a new group.
+     *
+     * @param group the Group to add
+     * @return the added Group
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws IllegalStateException if a group with the same name already exists
+     */
+    public final synchronized Group addGroup(Group group) throws AuthorizationAccessException {
+        return doAddGroup(group);
+    }
+
+    /**
+     * Adds a new group.
+     *
+     * @param group the Group to add
+     * @return the added Group
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract Group doAddGroup(Group group) throws AuthorizationAccessException;
+
+    /**
+     * Retrieves a Group by id.
+     *
+     * @param identifier the identifier of the Group to retrieve
+     * @return the Group with the given identifier, or null if no matching group was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract Group getGroup(String identifier) throws AuthorizationAccessException;
+
+    /**
+     * The group represented by the provided instance will be updated based on the provided instance.
+     *
+     * @param group an updated group instance
+     * @return the updated group instance, or null if no matching group was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws IllegalStateException if there is already a group with the same name
+     */
+    public final synchronized Group updateGroup(Group group) throws AuthorizationAccessException {
+        return doUpdateGroup(group);
+    }
+
+    /**
+     * The group represented by the provided instance will be updated based on the provided instance.
+     *
+     * @param group an updated group instance
+     * @return the updated group instance, or null if no matching group was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract Group doUpdateGroup(Group group) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the given group.
+     *
+     * @param group the group to delete
+     * @return the deleted group, or null if no matching group was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract Group deleteGroup(Group group) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the group with the given identifier.
+     *
+     * @param groupIdentifier the id of the group to delete
+     * @return the deleted group, or null if no matching group was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract Group deleteGroup(String groupIdentifier) throws AuthorizationAccessException;
+
+    /**
+     * Retrieves all groups.
+     *
+     * @return a list of groups
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract Set<Group> getGroups() throws AuthorizationAccessException;
+
+
+    /**
+     * Adds the given user.
+     *
+     * @param user the user to add
+     * @return the user that was added
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws IllegalStateException if there is already a user with the same identity
+     */
+    public final synchronized User addUser(User user) throws AuthorizationAccessException {
+        return doAddUser(user);
+    }
+
+    /**
+     * Adds the given user.
+     *
+     * @param user the user to add
+     * @return the user that was added
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract User doAddUser(User user) throws AuthorizationAccessException;
+
+    /**
+     * Retrieves the user with the given identifier.
+     *
+     * @param identifier the id of the user to retrieve
+     * @return the user with the given id, or null if no matching user was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract User getUser(String identifier) throws AuthorizationAccessException;
+
+    /**
+     * Retrieves the user with the given identity.
+     *
+     * @param identity the identity of the user to retrieve
+     * @return the user with the given identity, or null if no matching user was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract User getUserByIdentity(String identity) throws AuthorizationAccessException;
+
+    /**
+     * The user represented by the provided instance will be updated based on the provided instance.
+     *
+     * @param user an updated user instance
+     * @return the updated user instance, or null if no matching user was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws IllegalStateException if there is already a user with the same identity
+     */
+    public final synchronized User updateUser(final User user) throws AuthorizationAccessException {
+        return doUpdateUser(user);
+    }
+
+    /**
+     * The user represented by the provided instance will be updated based on the provided instance.
+     *
+     * @param user an updated user instance
+     * @return the updated user instance, or null if no matching user was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract User doUpdateUser(User user) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the given user.
+     *
+     * @param user the user to delete
+     * @return the user that was deleted, or null if no matching user was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract User deleteUser(User user) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the user with the given id.
+     *
+     * @param userIdentifier the identifier of the user to delete
+     * @return the user that was deleted, or null if no matching user was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract User deleteUser(String userIdentifier) throws AuthorizationAccessException;
+
+    /**
+     * Retrieves all users.
+     *
+     * @return a list of users
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract Set<User> getUsers() throws AuthorizationAccessException;
+
+    /**
+     * Adds the given policy ensuring that multiple policies can not be added for the same resource and action.
+     *
+     * @param accessPolicy the policy to add
+     * @return the policy that was added
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public final synchronized AccessPolicy addAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
+        return doAddAccessPolicy(accessPolicy);
+    }
+
+    /**
+     * Adds the given policy.
+     *
+     * @param accessPolicy the policy to add
+     * @return the policy that was added
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    protected abstract AccessPolicy doAddAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException;
+
+    /**
+     * Retrieves the policy with the given identifier.
+     *
+     * @param identifier the id of the policy to retrieve
+     * @return the policy with the given id, or null if no matching policy exists
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException;
+
+    /**
+     * The policy represented by the provided instance will be updated based on the provided instance.
+     *
+     * @param accessPolicy an updated policy
+     * @return the updated policy, or null if no matching policy was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract AccessPolicy updateAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the given policy.
+     *
+     * @param policy the policy to delete
+     * @return the deleted policy, or null if no matching policy was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract AccessPolicy deleteAccessPolicy(AccessPolicy policy) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the policy with the given id.
+     *
+     * @param policyIdentifier the id of the policy to delete
+     * @return the deleted policy, or null if no matching policy was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract AccessPolicy deleteAccessPolicy(String policyIdentifier) throws AuthorizationAccessException;
+
+    /**
+     * Retrieves all access policies.
+     *
+     * @return a list of policies
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException;
+
+    /**
+     * Returns the UserAccessPolicies instance.
+     *
+     * @return the UserAccessPolicies instance
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    public abstract UsersAndAccessPolicies getUsersAndAccessPolicies() throws AuthorizationAccessException;
+
+    /**
+     * Returns whether the proposed fingerprint is inheritable.
+     *
+     * @param proposedFingerprint the proposed fingerprint
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws UninheritableAuthorizationsException if the proposed fingerprint was uninheritable
+     */
+    @Override
+    public final void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
+        try {
+            // ensure we understand the proposed fingerprint
+            parsePoliciesUsersAndGroups(proposedFingerprint);
+        } catch (final AuthorizationAccessException e) {
+            throw new UninheritableAuthorizationsException("Unable to parse proposed fingerprint: " + e);
+        }
+
+        final List<User> users = getSortedUsers();
+        final List<Group> groups = getSortedGroups();
+        final List<AccessPolicy> accessPolicies = getSortedAccessPolicies();
+
+        // ensure we're in a state to inherit
+        if (!users.isEmpty() || !groups.isEmpty() || !accessPolicies.isEmpty()) {
+            throw new UninheritableAuthorizationsException("Proposed fingerprint is not inheritable because the current Authorizations is not empty..");
+        }
+    }
+
+    /**
+     * Parses the fingerprint and adds any users, groups, and policies to the current Authorizer.
+     *
+     * @param fingerprint the fingerprint that was obtained from calling getFingerprint() on another Authorizer.
+     */
+    @Override
+    public final void inheritFingerprint(final String fingerprint) throws AuthorizationAccessException {
+        if (fingerprint == null || fingerprint.trim().isEmpty()) {
+            return;
+        }
+
+        final PoliciesUsersAndGroups policiesUsersAndGroups = parsePoliciesUsersAndGroups(fingerprint);
+        policiesUsersAndGroups.getUsers().forEach(user -> addUser(user));
+        policiesUsersAndGroups.getGroups().forEach(group -> addGroup(group));
+        policiesUsersAndGroups.getAccessPolicies().forEach(policy -> addAccessPolicy(policy));
+    }
+
+    private PoliciesUsersAndGroups parsePoliciesUsersAndGroups(final String fingerprint) {
+        final List<AccessPolicy> accessPolicies = new ArrayList<>();
+        final List<User> users = new ArrayList<>();
+        final List<Group> groups = new ArrayList<>();
+
+        final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
+        try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
+            final DocumentBuilder docBuilder = DOCUMENT_BUILDER_FACTORY.newDocumentBuilder();
+            final Document document = docBuilder.parse(in);
+            final Element rootElement = document.getDocumentElement();
+
+            // parse all the users and add them to the current authorizer
+            NodeList userNodes = rootElement.getElementsByTagName(USER_ELEMENT);
+            for (int i=0; i < userNodes.getLength(); i++) {
+                Node userNode = userNodes.item(i);
+                users.add(parseUser((Element) userNode));
+            }
+
+            // parse all the groups and add them to the current authorizer
+            NodeList groupNodes = rootElement.getElementsByTagName(GROUP_ELEMENT);
+            for (int i=0; i < groupNodes.getLength(); i++) {
+                Node groupNode = groupNodes.item(i);
+                groups.add(parseGroup((Element) groupNode));
+            }
+
+            // parse all the policies and add them to the current authorizer
+            NodeList policyNodes = rootElement.getElementsByTagName(POLICY_ELEMENT);
+            for (int i=0; i < policyNodes.getLength(); i++) {
+                Node policyNode = policyNodes.item(i);
+                accessPolicies.add(parsePolicy((Element) policyNode));
+            }
+        } catch (SAXException | ParserConfigurationException | IOException e) {
+            throw new AuthorizationAccessException("Unable to parse fingerprint", e);
+        }
+
+        return new PoliciesUsersAndGroups(accessPolicies, users, groups);
+    }
+
+    private User parseUser(final Element element) {
+        final User.Builder builder = new User.Builder()
+                .identifier(element.getAttribute(IDENTIFIER_ATTR))
+                .identity(element.getAttribute(IDENTITY_ATTR));
+
+        return builder.build();
+    }
+
+    private Group parseGroup(final Element element) {
+        final Group.Builder builder = new Group.Builder()
+                .identifier(element.getAttribute(IDENTIFIER_ATTR))
+                .name(element.getAttribute(NAME_ATTR));
+
+        NodeList groupUsers = element.getElementsByTagName(GROUP_USER_ELEMENT);
+        for (int i=0; i < groupUsers.getLength(); i++) {
+            Element groupUserNode = (Element) groupUsers.item(i);
+            builder.addUser(groupUserNode.getAttribute(IDENTIFIER_ATTR));
+        }
+
+        return builder.build();
+    }
+
+    private AccessPolicy parsePolicy(final Element element) {
+        final AccessPolicy.Builder builder = new AccessPolicy.Builder()
+                .identifier(element.getAttribute(IDENTIFIER_ATTR))
+                .resource(element.getAttribute(RESOURCE_ATTR));
+
+        final String actions = element.getAttribute(ACTIONS_ATTR);
+        if (actions.equals(RequestAction.READ.name())) {
+            builder.action(RequestAction.READ);
+        } else if (actions.equals(RequestAction.WRITE.name())) {
+            builder.action(RequestAction.WRITE);
+        } else if (actions.equals(RequestAction.DELETE.name())) {
+            builder.action(RequestAction.DELETE);
+        } else {
+            throw new IllegalStateException("Unknown Policy Action: " + actions);
+        }
+
+        NodeList policyUsers = element.getElementsByTagName(POLICY_USER_ELEMENT);
+        for (int i=0; i < policyUsers.getLength(); i++) {
+            Element policyUserNode = (Element) policyUsers.item(i);
+            builder.addUser(policyUserNode.getAttribute(IDENTIFIER_ATTR));
+        }
+
+        NodeList policyGroups = element.getElementsByTagName(POLICY_GROUP_ELEMENT);
+        for (int i=0; i < policyGroups.getLength(); i++) {
+            Element policyGroupNode = (Element) policyGroups.item(i);
+            builder.addGroup(policyGroupNode.getAttribute(IDENTIFIER_ATTR));
+        }
+
+        return builder.build();
+    }
+
+    @Override
+    public final AccessPolicyProvider getAccessPolicyProvider() {
+        return new ConfigurableAccessPolicyProvider() {
+            @Override
+            public Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException {
+                return AbstractPolicyBasedAuthorizer.this.getAccessPolicies();
+            }
+
+            @Override
+            public AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException {
+                return AbstractPolicyBasedAuthorizer.this.getAccessPolicy(identifier);
+            }
+
+            @Override
+            public AccessPolicy addAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
+                return AbstractPolicyBasedAuthorizer.this.addAccessPolicy(accessPolicy);
+            }
+
+            @Override
+            public AccessPolicy updateAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
+                return AbstractPolicyBasedAuthorizer.this.updateAccessPolicy(accessPolicy);
+            }
+
+            @Override
+            public AccessPolicy deleteAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException {
+                return AbstractPolicyBasedAuthorizer.this.deleteAccessPolicy(accessPolicy);
+            }
+
+            @Override
+            public AccessPolicy deleteAccessPolicy(String accessPolicyIdentifier) throws AuthorizationAccessException {
+                return AbstractPolicyBasedAuthorizer.this.deleteAccessPolicy(accessPolicyIdentifier);
+            }
+
+            @Override
+            public AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) throws AuthorizationAccessException {
+                final UsersAndAccessPolicies usersAndAccessPolicies = AbstractPolicyBasedAuthorizer.this.getUsersAndAccessPolicies();
+                return usersAndAccessPolicies.getAccessPolicy(resourceIdentifier, action);
+            }
+
+            @Override
+            public String getFingerprint() throws AuthorizationAccessException {
+                // fingerprint is managed by the encapsulating class
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
+                // fingerprint is managed by the encapsulating class
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
+                // fingerprint is managed by the encapsulating class
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
+            public UserGroupProvider getUserGroupProvider() {
+                return new ConfigurableUserGroupProvider() {
+                    @Override
+                    public User addUser(User user) throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.addUser(user);
+                    }
+
+                    @Override
+                    public User updateUser(User user) throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.updateUser(user);
+                    }
+
+                    @Override
+                    public User deleteUser(User user) throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.deleteUser(user);
+                    }
+
+                    @Override
+                    public User deleteUser(String userIdentifier) throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.deleteUser(userIdentifier);
+                    }
+
+                    @Override
+                    public Group addGroup(Group group) throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.addGroup(group);
+                    }
+
+                    @Override
+                    public Group updateGroup(Group group) throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.updateGroup(group);
+                    }
+
+                    @Override
+                    public Group deleteGroup(Group group) throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.deleteGroup(group);
+                    }
+
+                    @Override
+                    public Group deleteGroup(String groupIdentifier) throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.deleteGroup(groupIdentifier);
+                    }
+
+                    @Override
+                    public Set<User> getUsers() throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.getUsers();
+                    }
+
+                    @Override
+                    public User getUser(String identifier) throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.getUser(identifier);
+                    }
+
+                    @Override
+                    public User getUserByIdentity(String identity) throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.getUserByIdentity(identity);
+                    }
+
+                    @Override
+                    public Set<Group> getGroups() throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.getGroups();
+                    }
+
+                    @Override
+                    public Group getGroup(String identifier) throws AuthorizationAccessException {
+                        return AbstractPolicyBasedAuthorizer.this.getGroup(identifier);
+                    }
+
+                    @Override
+                    public UserAndGroups getUserAndGroups(String identity) throws AuthorizationAccessException {
+                        final UsersAndAccessPolicies usersAndAccessPolicies = AbstractPolicyBasedAuthorizer.this.getUsersAndAccessPolicies();
+                        final User user = usersAndAccessPolicies.getUser(identity);
+                        final Set<Group> groups = usersAndAccessPolicies.getGroups(identity);
+
+                        return new UserAndGroups() {
+                            @Override
+                            public User getUser() {
+                                return user;
+                            }
+
+                            @Override
+                            public Set<Group> getGroups() {
+                                return groups;
+                            }
+                        };
+                    }
+
+                    @Override
+                    public String getFingerprint() throws AuthorizationAccessException {
+                        // fingerprint is managed by the encapsulating class
+                        throw new UnsupportedOperationException();
+                    }
+
+                    @Override
+                    public void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
+                        // fingerprint is managed by the encapsulating class
+                        throw new UnsupportedOperationException();
+                    }
+
+                    @Override
+                    public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException {
+                        // fingerprint is managed by the encapsulating class
+                        throw new UnsupportedOperationException();
+                    }
+
+                    @Override
+                    public void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
+                    }
+
+                    @Override
+                    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+                    }
+
+                    @Override
+                    public void preDestruction() throws AuthorizerDestructionException {
+                    }
+                };
+            }
+
+            @Override
+            public void initialize(AccessPolicyProviderInitializationContext initializationContext) throws AuthorizerCreationException {
+            }
+
+            @Override
+            public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+            }
+
+            @Override
+            public void preDestruction() throws AuthorizerDestructionException {
+            }
+        };
+    }
+
+    /**
+     * Returns a fingerprint representing the authorizations managed by this authorizer. The fingerprint will be
+     * used for comparison to determine if two policy-based authorizers represent a compatible set of users,
+     * groups, and policies.
+     *
+     * @return the fingerprint for this Authorizer
+     */
+    @Override
+    public final String getFingerprint() throws AuthorizationAccessException {
+        final List<User> users = getSortedUsers();
+        final List<Group> groups = getSortedGroups();
+        final List<AccessPolicy> policies = getSortedAccessPolicies();
+
+        XMLStreamWriter writer = null;
+        final StringWriter out = new StringWriter();
+        try {
+            writer = XML_OUTPUT_FACTORY.createXMLStreamWriter(out);
+            writer.writeStartDocument();
+            writer.writeStartElement("authorizations");
+
+            for (User user : users) {
+                writeUser(writer, user);
+            }
+            for (Group group : groups) {
+                writeGroup(writer, group);
+            }
+            for (AccessPolicy policy : policies) {
+                writePolicy(writer, policy);
+            }
+
+            writer.writeEndElement();
+            writer.writeEndDocument();
+            writer.flush();
+        } catch (XMLStreamException e) {
+            throw new AuthorizationAccessException("Unable to generate fingerprint", e);
+        } finally {
+            if (writer != null) {
+                try {
+                    writer.close();
+                } catch (XMLStreamException e) {
+                    // nothing to do here
+                }
+            }
+        }
+
+        return out.toString();
+    }
+
+    private void writeUser(final XMLStreamWriter writer, final User user) throws XMLStreamException {
+        writer.writeStartElement(USER_ELEMENT);
+        writer.writeAttribute(IDENTIFIER_ATTR, user.getIdentifier());
+        writer.writeAttribute(IDENTITY_ATTR, user.getIdentity());
+        writer.writeEndElement();
+    }
+
+    private void writeGroup(final XMLStreamWriter writer, final Group group) throws XMLStreamException {
+        List<String> users = new ArrayList<>(group.getUsers());
+        Collections.sort(users);
+
+        writer.writeStartElement(GROUP_ELEMENT);
+        writer.writeAttribute(IDENTIFIER_ATTR, group.getIdentifier());
+        writer.writeAttribute(NAME_ATTR, group.getName());
+
+        for (String user : users) {
+            writer.writeStartElement(GROUP_USER_ELEMENT);
+            writer.writeAttribute(IDENTIFIER_ATTR, user);
+            writer.writeEndElement();
+        }
+
+        writer.writeEndElement();
+    }
+
+    private void writePolicy(final XMLStreamWriter writer, final AccessPolicy policy) throws XMLStreamException {
+        // sort the users for the policy
+        List<String> policyUsers = new ArrayList<>(policy.getUsers());
+        Collections.sort(policyUsers);
+
+        // sort the groups for this policy
+        List<String> policyGroups = new ArrayList<>(policy.getGroups());
+        Collections.sort(policyGroups);
+
+        writer.writeStartElement(POLICY_ELEMENT);
+        writer.writeAttribute(IDENTIFIER_ATTR, policy.getIdentifier());
+        writer.writeAttribute(RESOURCE_ATTR, policy.getResource());
+        writer.writeAttribute(ACTIONS_ATTR, policy.getAction().name());
+
+        for (String policyUser : policyUsers) {
+            writer.writeStartElement(POLICY_USER_ELEMENT);
+            writer.writeAttribute(IDENTIFIER_ATTR, policyUser);
+            writer.writeEndElement();
+        }
+
+        for (String policyGroup : policyGroups) {
+            writer.writeStartElement(POLICY_GROUP_ELEMENT);
+            writer.writeAttribute(IDENTIFIER_ATTR, policyGroup);
+            writer.writeEndElement();
+        }
+
+        writer.writeEndElement();
+    }
+
+    private List<AccessPolicy> getSortedAccessPolicies() {
+        final List<AccessPolicy> policies = new ArrayList<>(getAccessPolicies());
+        Collections.sort(policies, Comparator.comparing(AccessPolicy::getIdentifier));
+        return policies;
+    }
+
+    private List<Group> getSortedGroups() {
+        final List<Group> groups = new ArrayList<>(getGroups());
+        Collections.sort(groups, Comparator.comparing(Group::getIdentifier));
+        return groups;
+    }
+
+    private List<User> getSortedUsers() {
+        final List<User> users = new ArrayList<>(getUsers());
+        Collections.sort(users, Comparator.comparing(User::getIdentifier));
+        return users;
+    }
+
+    private static class PoliciesUsersAndGroups {
+        final List<AccessPolicy> accessPolicies;
+        final List<User> users;
+        final List<Group> groups;
+
+        public PoliciesUsersAndGroups(List<AccessPolicy> accessPolicies, List<User> users, List<Group> groups) {
+            this.accessPolicies = accessPolicies;
+            this.users = users;
+            this.groups = groups;
+        }
+
+        public List<AccessPolicy> getAccessPolicies() {
+            return accessPolicies;
+        }
+
+        public List<User> getUsers() {
+            return users;
+        }
+
+        public List<Group> getGroups() {
+            return groups;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizableLookup.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizableLookup.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizableLookup.java
new file mode 100644
index 0000000..f5bbd4d
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizableLookup.java
@@ -0,0 +1,83 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.registry.security.authorization.resource.Authorizable;
+
+public interface AuthorizableLookup {
+
+    /**
+     * Get the authorizable for retrieving resources.
+     *
+     * @return authorizable
+     */
+    Authorizable getResourcesAuthorizable();
+
+    /**
+     * Get the authorizable for /proxy.
+     *
+     * @return authorizable
+     */
+    Authorizable getProxyAuthorizable();
+
+    /**
+     * Get the authorizable for all tenants.
+     *
+     * Get the {@link Authorizable} that represents the resource of users and user groups.
+     * @return authorizable
+     */
+    Authorizable getTenantsAuthorizable();
+
+    /**
+     * Get the authorizable for all access policies.
+     *
+     * @return authorizable
+     */
+    Authorizable getPoliciesAuthorizable();
+
+    /**
+     * Get the authorizable for all Buckets.
+     *
+     * @return authorizable
+     */
+    Authorizable getBucketsAuthorizable();
+
+    /**
+     * Get the authorizable for the Bucket with the bucket id.
+     *
+     * @param bucketIdentifier bucket id
+     * @return authorizable
+     */
+    Authorizable getBucketAuthorizable(String bucketIdentifier);
+
+    /**
+     * Get the authorizable for the policy of the specified resource.
+     *
+     * @param resource resource
+     * @return authorizable
+     */
+    Authorizable getAccessPolicyByResource(String resource);
+
+    /**
+     * Get the authorizable of the specified resource.
+     *
+     * @param resource resource
+     * @return authorizable
+     */
+    Authorizable getAuthorizableByResource(final String resource);
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizeAccess.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizeAccess.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizeAccess.java
new file mode 100644
index 0000000..94dc3be
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizeAccess.java
@@ -0,0 +1,21 @@
+/*
+ * 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.authorization;
+
+public interface AuthorizeAccess {
+    void authorize(AuthorizableLookup lookup);
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerCapabilityDetection.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerCapabilityDetection.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerCapabilityDetection.java
new file mode 100644
index 0000000..5252c7f
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerCapabilityDetection.java
@@ -0,0 +1,84 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.registry.security.authorization.AccessPolicy;
+import org.apache.nifi.registry.security.authorization.AccessPolicyProvider;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.ConfigurableAccessPolicyProvider;
+import org.apache.nifi.registry.security.authorization.ConfigurableUserGroupProvider;
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.ManagedAuthorizer;
+import org.apache.nifi.registry.security.authorization.User;
+
+public final class AuthorizerCapabilityDetection {
+
+    public static boolean isManagedAuthorizer(final Authorizer authorizer) {
+        return authorizer instanceof ManagedAuthorizer;
+    }
+
+    public static boolean isConfigurableAccessPolicyProvider(final Authorizer authorizer) {
+        if (!isManagedAuthorizer(authorizer)) {
+            return false;
+        }
+
+        final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
+        return managedAuthorizer.getAccessPolicyProvider() instanceof ConfigurableAccessPolicyProvider;
+    }
+
+    public static boolean isConfigurableUserGroupProvider(final Authorizer authorizer) {
+        if (!isManagedAuthorizer(authorizer)) {
+            return false;
+        }
+
+        final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
+        final AccessPolicyProvider accessPolicyProvider = managedAuthorizer.getAccessPolicyProvider();
+        return accessPolicyProvider.getUserGroupProvider() instanceof ConfigurableUserGroupProvider;
+    }
+
+    public static boolean isUserConfigurable(final Authorizer authorizer, final User user) {
+        if (!isConfigurableUserGroupProvider(authorizer)) {
+            return false;
+        }
+
+        final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
+        final ConfigurableUserGroupProvider configurableUserGroupProvider = (ConfigurableUserGroupProvider) managedAuthorizer.getAccessPolicyProvider().getUserGroupProvider();
+        return configurableUserGroupProvider.isConfigurable(user);
+    }
+
+    public static boolean isGroupConfigurable(final Authorizer authorizer, final Group group) {
+        if (!isConfigurableUserGroupProvider(authorizer)) {
+            return false;
+        }
+
+        final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
+        final ConfigurableUserGroupProvider configurableUserGroupProvider = (ConfigurableUserGroupProvider) managedAuthorizer.getAccessPolicyProvider().getUserGroupProvider();
+        return configurableUserGroupProvider.isConfigurable(group);
+    }
+
+    public static boolean isAccessPolicyConfigurable(final Authorizer authorizer, final AccessPolicy accessPolicy) {
+        if (!isConfigurableAccessPolicyProvider(authorizer)) {
+            return false;
+        }
+
+        final ManagedAuthorizer managedAuthorizer = (ManagedAuthorizer) authorizer;
+        final ConfigurableAccessPolicyProvider configurableAccessPolicyProvider = (ConfigurableAccessPolicyProvider) managedAuthorizer.getAccessPolicyProvider();
+        return configurableAccessPolicyProvider.isConfigurable(accessPolicy);
+    }
+
+    private AuthorizerCapabilityDetection() {}
+}


[06/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Group.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Group.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Group.java
deleted file mode 100644
index f22dd97..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Group.java
+++ /dev/null
@@ -1,263 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Objects;
-import java.util.Set;
-import java.util.UUID;
-
-/**
- * A group that users can belong to.
- */
-public class Group {
-
-    private final String identifier;
-
-    private final String name;
-
-    private final Set<String> users;
-
-    private Group(final Builder builder) {
-        this.identifier = builder.identifier;
-        this.name = builder.name;
-        this.users = Collections.unmodifiableSet(new HashSet<>(builder.users));
-
-        if (this.identifier == null || this.identifier.trim().isEmpty()) {
-            throw new IllegalArgumentException("Identifier can not be null or empty");
-        }
-
-        if (this.name == null || this.name.trim().isEmpty()) {
-            throw new IllegalArgumentException("Name can not be null or empty");
-        }
-    }
-
-    /**
-     * @return the identifier of the group
-     */
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    /**
-     * @return the name of the group
-     */
-    public String getName() {
-        return name;
-    }
-
-    /**
-     * @return an unmodifiable set of user identifiers that belong to this group
-     */
-    public Set<String> getUsers() {
-        return users;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-
-        final Group other = (Group) obj;
-        return Objects.equals(this.identifier, other.identifier);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(this.identifier);
-    }
-
-    @Override
-    public String toString() {
-        return String.format("identifier[%s], name[%s]", getIdentifier(), getName());
-    }
-
-
-    /**
-     * Builder for creating Groups.
-     */
-    public static class Builder {
-
-        private String identifier;
-        private String name;
-        private Set<String> users = new HashSet<>();
-        private final boolean fromGroup;
-
-        public Builder() {
-            this.fromGroup = false;
-        }
-
-        /**
-         * Initializes the builder with the state of the provided group. When using this constructor
-         * the identifier field of the builder can not be changed and will result in an IllegalStateException
-         * if attempting to do so.
-         *
-         * @param other the existing access policy to initialize from
-         */
-        public Builder(final Group other) {
-            if (other == null) {
-                throw new IllegalArgumentException("Provided group can not be null");
-            }
-
-            this.identifier = other.getIdentifier();
-            this.name = other.getName();
-            this.users.clear();
-            this.users.addAll(other.getUsers());
-            this.fromGroup = true;
-        }
-
-        /**
-         * Sets the identifier of the builder.
-         *
-         * @param identifier the identifier
-         * @return the builder
-         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Group
-         */
-        public Builder identifier(final String identifier) {
-            if (fromGroup) {
-                throw new IllegalStateException(
-                        "Identifier can not be changed when initialized from an existing group");
-            }
-
-            this.identifier = identifier;
-            return this;
-        }
-
-        /**
-         * Sets the identifier of the builder to a random UUID.
-         *
-         * @return the builder
-         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Group
-         */
-        public Builder identifierGenerateRandom() {
-            if (fromGroup) {
-                throw new IllegalStateException(
-                        "Identifier can not be changed when initialized from an existing group");
-            }
-
-            this.identifier = UUID.randomUUID().toString();
-            return this;
-        }
-
-        /**
-         * Sets the identifier of the builder with a UUID generated from the specified seed string.
-         *
-         * @return the builder
-         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Group
-         */
-        public Builder identifierGenerateFromSeed(final String seed) {
-            if (fromGroup) {
-                throw new IllegalStateException(
-                        "Identifier can not be changed when initialized from an existing group");
-            }
-            if (seed == null) {
-                throw new IllegalArgumentException("Cannot seed the group identifier with a null value.");
-            }
-
-            this.identifier = UUID.nameUUIDFromBytes(seed.getBytes(StandardCharsets.UTF_8)).toString();
-            return this;
-        }
-
-        /**
-         * Sets the name of the builder.
-         *
-         * @param name the name
-         * @return the builder
-         */
-        public Builder name(final String name) {
-            this.name = name;
-            return this;
-        }
-
-        /**
-         * Adds all users from the provided set to the builder's set of users.
-         *
-         * @param users a set of users to add
-         * @return the builder
-         */
-        public Builder addUsers(final Set<String> users) {
-            if (users != null) {
-                this.users.addAll(users);
-            }
-            return this;
-        }
-
-        /**
-         * Adds the given user to the builder's set of users.
-         *
-         * @param user the user to add
-         * @return the builder
-         */
-        public Builder addUser(final String user) {
-            if (user != null) {
-                this.users.add(user);
-            }
-            return this;
-        }
-
-        /**
-         * Removes the given user from the builder's set of users.
-         *
-         * @param user the user to remove
-         * @return the builder
-         */
-        public Builder removeUser(final String user) {
-            if (user != null) {
-                this.users.remove(user);
-            }
-            return this;
-        }
-
-        /**
-         * Removes all users from the provided set from the builder's set of users.
-         *
-         * @param users the users to remove
-         * @return the builder
-         */
-        public Builder removeUsers(final Set<String> users) {
-            if (users != null) {
-                this.users.removeAll(users);
-            }
-            return this;
-        }
-
-        /**
-         * Clears the builder's set of users so that users is non-null with size 0.
-         *
-         * @return the builder
-         */
-        public Builder clearUsers() {
-            this.users.clear();
-            return this;
-        }
-
-        /**
-         * @return a new Group constructed from the state of the builder
-         */
-        public Group build() {
-            return new Group(this);
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ManagedAuthorizer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ManagedAuthorizer.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ManagedAuthorizer.java
deleted file mode 100644
index da82f4e..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/ManagedAuthorizer.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.UninheritableAuthorizationsException;
-
-public interface ManagedAuthorizer extends Authorizer {
-
-    /**
-     * Returns a fingerprint representing the authorizations managed by this authorizer. The fingerprint will be
-     * used for comparison to determine if two managed authorizers represent a compatible set of users,
-     * groups, and/or policies. Must be non null
-     *
-     * @return the fingerprint for this Authorizer
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    String getFingerprint() throws AuthorizationAccessException;
-
-    /**
-     * Parses the fingerprint and adds any users, groups, and policies to the current Authorizer.
-     *
-     * @param fingerprint the fingerprint that was obtained from calling getFingerprint() on another Authorizer.
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    void inheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
-
-    /**
-     * When the fingerprints are not equal, this method will check if the proposed fingerprint is inheritable.
-     * If the fingerprint is an exact match, this method will not be invoked as there is nothing to inherit.
-     *
-     * @param proposedFingerprint the proposed fingerprint
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     * @throws UninheritableAuthorizationsException if the proposed fingerprint was uninheritable
-     */
-    void checkInheritability(final String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException;
-
-    /**
-     * Returns the AccessPolicy provider for this managed Authorizer. Must be non null
-     *
-     * @return the AccessPolicy provider
-     */
-    AccessPolicyProvider getAccessPolicyProvider();
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/RequestAction.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/RequestAction.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/RequestAction.java
deleted file mode 100644
index a489ecc..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/RequestAction.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import java.util.StringJoiner;
-
-/**
- * Actions a user/entity can take on a resource.
- */
-public enum RequestAction {
-    READ("read"),
-    WRITE("write"),
-    DELETE("delete");
-
-    private String value;
-
-    RequestAction(String value) {
-        this.value = value;
-    }
-
-    @Override
-    public String toString() {
-        return value.toLowerCase();
-    }
-
-    public static RequestAction valueOfValue(final String action) {
-        if (RequestAction.READ.toString().equalsIgnoreCase(action)) {
-            return RequestAction.READ;
-        } else if (RequestAction.WRITE.toString().equalsIgnoreCase(action)) {
-            return RequestAction.WRITE;
-        } else if (RequestAction.DELETE.toString().equalsIgnoreCase(action)) {
-            return RequestAction.DELETE;
-        } else {
-            StringJoiner stringJoiner = new StringJoiner(", ");
-            for(RequestAction ra : RequestAction.values()) {
-                stringJoiner.add(ra.toString());
-            }
-            String allowableValues = stringJoiner.toString();
-            throw new IllegalArgumentException("Action must be one of [" + allowableValues + "]");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Resource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Resource.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Resource.java
deleted file mode 100644
index 711f724..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/Resource.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-/**
- * Resource in an authorization request.
- */
-public interface Resource {
-
-    /**
-     * The identifier for this resource.
-     *
-     * @return identifier for this resource
-     */
-    String getIdentifier();
-
-    /**
-     * The name of this resource. May be null.
-     *
-     * @return name of this resource
-     */
-    String getName();
-
-    /**
-     * The description of this resource that may be safely used in messages to the client.
-     *
-     * @return safe description
-     */
-    String getSafeDescription();
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/User.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/User.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/User.java
deleted file mode 100644
index 79f12a8..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/User.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import java.nio.charset.StandardCharsets;
-import java.util.Objects;
-import java.util.UUID;
-
-/**
- * A user to create authorization policies for.
- */
-public class User {
-
-    private final String identifier;
-
-    private final String identity;
-
-    private User(final Builder builder) {
-        this.identifier = builder.identifier;
-        this.identity = builder.identity;
-
-        if (identifier == null || identifier.trim().isEmpty()) {
-            throw new IllegalArgumentException("Identifier can not be null or empty");
-        }
-
-        if (identity == null || identity.trim().isEmpty()) {
-            throw new IllegalArgumentException("Identity can not be null or empty");
-        }
-
-    }
-
-    /**
-     * @return the identifier of the user
-     */
-    public String getIdentifier() {
-        return identifier;
-    }
-
-    /**
-     * @return the identity string of the user
-     */
-    public String getIdentity() {
-        return identity;
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj == null) {
-            return false;
-        }
-        if (getClass() != obj.getClass()) {
-            return false;
-        }
-
-        final User other = (User) obj;
-        return Objects.equals(this.identifier, other.identifier);
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(this.identifier);
-    }
-
-    @Override
-    public String toString() {
-        return String.format("identifier[%s], identity[%s]", getIdentifier(), getIdentity());
-    }
-
-    /**
-     * Builder for Users.
-     */
-    public static class Builder {
-
-        private String identifier;
-        private String identity;
-        private final boolean fromUser;
-
-        /**
-         * Default constructor for building a new User.
-         */
-        public Builder() {
-            this.fromUser = false;
-        }
-
-        /**
-         * Initializes the builder with the state of the provided user. When using this constructor
-         * the identifier field of the builder can not be changed and will result in an IllegalStateException
-         * if attempting to do so.
-         *
-         * @param other the existing user to initialize from
-         */
-        public Builder(final User other) {
-            if (other == null) {
-                throw new IllegalArgumentException("Provided user can not be null");
-            }
-
-            this.identifier = other.getIdentifier();
-            this.identity = other.getIdentity();
-            this.fromUser = true;
-        }
-
-        /**
-         * Sets the identifier of the builder.
-         *
-         * @param identifier the identifier to set
-         * @return the builder
-         * @throws IllegalStateException if this method is called when this builder was constructed from an existing User
-         */
-        public Builder identifier(final String identifier) {
-            if (fromUser) {
-                throw new IllegalStateException(
-                        "Identifier can not be changed when initialized from an existing user");
-            }
-
-            this.identifier = identifier;
-            return this;
-        }
-
-        /**
-         * Sets the identifier of the builder to a random UUID.
-         *
-         * @return the builder
-         * @throws IllegalStateException if this method is called when this builder was constructed from an existing User
-         */
-        public Builder identifierGenerateRandom() {
-            if (fromUser) {
-                throw new IllegalStateException(
-                        "Identifier can not be changed when initialized from an existing user");
-            }
-
-            this.identifier = UUID.randomUUID().toString();
-            return this;
-        }
-
-        /**
-         * Sets the identifier of the builder with a UUID generated from the specified seed string.
-         *
-         * @return the builder
-         * @throws IllegalStateException if this method is called when this builder was constructed from an existing User
-         */
-        public Builder identifierGenerateFromSeed(final String seed) {
-            if (fromUser) {
-                throw new IllegalStateException(
-                        "Identifier can not be changed when initialized from an existing user");
-            }
-            if (seed == null) {
-                throw new IllegalArgumentException("Cannot seed the user identifier with a null value.");
-            }
-
-            this.identifier = UUID.nameUUIDFromBytes(seed.getBytes(StandardCharsets.UTF_8)).toString();
-            return this;
-        }
-
-        /**
-         * Sets the identity of the builder.
-         *
-         * @param identity the identity to set
-         * @return the builder
-         */
-        public Builder identity(final String identity) {
-            this.identity = identity;
-            return this;
-        }
-
-        /**
-         * @return a new User constructed from the state of the builder
-         */
-        public User build() {
-            return new User(this);
-        }
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/UserAndGroups.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/UserAndGroups.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/UserAndGroups.java
deleted file mode 100644
index b8f150a..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/UserAndGroups.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import java.util.Set;
-
-/**
- * A holder object to provide atomic access to a user and their groups.
- */
-public interface UserAndGroups {
-
-    /**
-     * Retrieves the user, or null if the user is unknown
-     *
-     * @return the user with the given identity
-     */
-    User getUser();
-
-    /**
-     * Retrieves the groups for the user, or null if the user is unknown or has no groups.
-     *
-     * @return the set of groups for the given user identity
-     */
-    Set<Group> getGroups();
-
-}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/UserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/UserGroupProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/UserGroupProvider.java
deleted file mode 100644
index c0460da..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/UserGroupProvider.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization;
-
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerCreationException;
-import org.apache.nifi.registry.authorization.exception.AuthorizerDestructionException;
-
-import java.util.Set;
-
-/**
- * Provides access to Users and Groups.
- *
- * NOTE: Extensions will be called often and frequently. Because of this, if the underlying implementation needs to
- * make remote calls or expensive calculations those should probably be done asynchronously and/or cache the results.
- *
- * Additionally, extensions need to be thread safe.
- */
-public interface UserGroupProvider {
-
-    /**
-     * Retrieves all users. Must be non null
-     *
-     * @return a list of users
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    Set<User> getUsers() throws AuthorizationAccessException;
-
-    /**
-     * Retrieves the user with the given identifier.
-     *
-     * @param identifier the id of the user to retrieve
-     * @return the user with the given id, or null if no matching user was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    User getUser(String identifier) throws AuthorizationAccessException;
-
-    /**
-     * Retrieves the user with the given identity.
-     *
-     * @param identity the identity of the user to retrieve
-     * @return the user with the given identity, or null if no matching user was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    User getUserByIdentity(String identity) throws AuthorizationAccessException;
-
-    /**
-     * Retrieves all groups. Must be non null
-     *
-     * @return a list of groups
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    Set<Group> getGroups() throws AuthorizationAccessException;
-
-    /**
-     * Retrieves a Group by id.
-     *
-     * @param identifier the identifier of the Group to retrieve
-     * @return the Group with the given identifier, or null if no matching group was found
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    Group getGroup(String identifier) throws AuthorizationAccessException;
-
-    /**
-     * Gets a user and their groups. Must be non null. If the user is not known the UserAndGroups.getUser() and
-     * UserAndGroups.getGroups() should return null
-     *
-     * @return the UserAndGroups for the specified identity
-     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
-     */
-    UserAndGroups getUserAndGroups(String identity) throws AuthorizationAccessException;
-
-    /**
-     * Called immediately after instance creation for implementers to perform additional setup
-     *
-     * @param initializationContext in which to initialize
-     */
-    void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException;
-
-    /**
-     * Called to configure the Authorizer.
-     *
-     * @param configurationContext at the time of configuration
-     * @throws AuthorizerCreationException for any issues configuring the provider
-     */
-    void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException;
-
-    /**
-     * Called immediately before instance destruction for implementers to release resources.
-     *
-     * @throws AuthorizerDestructionException If pre-destruction fails.
-     */
-    void preDestruction() throws AuthorizerDestructionException;
-}

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/exception/AccessDeniedException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/exception/AccessDeniedException.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/exception/AccessDeniedException.java
deleted file mode 100644
index 7b09a6e..0000000
--- a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/authorization/exception/AccessDeniedException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.authorization.exception;
-
-/**
- * Represents any error that might occur while authorizing user requests.
- */
-public class AccessDeniedException extends RuntimeException {
-    private static final long serialVersionUID = -5683444815269084134L;
-
-    public AccessDeniedException(Throwable cause) {
-        super(cause);
-    }
-
-    public AccessDeniedException(String message, Throwable cause) {
-        super(message, cause);
-    }
-
-    public AccessDeniedException(String message) {
-        super(message);
-    }
-
-    public AccessDeniedException() {
-    }
-}

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicy.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicy.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicy.java
new file mode 100644
index 0000000..aa8260b
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicy.java
@@ -0,0 +1,367 @@
+/*
+ * 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.authorization;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ * Defines a policy for a set of userIdentifiers to perform a set of actions on a given resource.
+ */
+public class AccessPolicy {
+
+    private final String identifier;
+
+    private final String resource;
+
+    private final Set<String> users;
+
+    private final Set<String> groups;
+
+    private final RequestAction action;
+
+    private AccessPolicy(final Builder builder) {
+        this.identifier = builder.identifier;
+        this.resource = builder.resource;
+        this.action = builder.action;
+        this.users = Collections.unmodifiableSet(new HashSet<>(builder.users));
+        this.groups = Collections.unmodifiableSet(new HashSet<>(builder.groups));
+
+        if (this.identifier == null || this.identifier.trim().isEmpty()) {
+            throw new IllegalArgumentException("Identifier can not be null or empty");
+        }
+
+        if (this.resource == null) {
+            throw new IllegalArgumentException("Resource can not be null");
+        }
+
+        if (this.action == null) {
+            throw new IllegalArgumentException("Action can not be null");
+        }
+    }
+
+    /**
+     * @return the identifier for this policy
+     */
+    public String getIdentifier() {
+        return identifier;
+    }
+
+    /**
+     * @return the resource for this policy
+     */
+    public String getResource() {
+        return resource;
+    }
+
+    /**
+     * @return an unmodifiable set of user ids for this policy
+     */
+    public Set<String> getUsers() {
+        return users;
+    }
+
+    /**
+     * @return an unmodifiable set of group ids for this policy
+     */
+    public Set<String> getGroups() {
+        return groups;
+    }
+
+    /**
+     * @return the action for this policy
+     */
+    public RequestAction getAction() {
+        return action;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+
+        final AccessPolicy other = (AccessPolicy) obj;
+        return Objects.equals(this.identifier, other.identifier);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(this.identifier);
+    }
+
+    @Override
+    public String toString() {
+        return String.format("identifier[%s], resource[%s], users[%s], groups[%s], action[%s]",
+                getIdentifier(), getResource(), getUsers(), getGroups(), getAction());
+    }
+
+    /**
+     * Builder for Access Policies.
+     */
+    public static class Builder {
+
+        private String identifier;
+        private String resource;
+        private RequestAction action;
+        private Set<String> users = new HashSet<>();
+        private Set<String> groups = new HashSet<>();
+        private final boolean fromPolicy;
+
+        /**
+         * Default constructor for building a new AccessPolicy.
+         */
+        public Builder() {
+            this.fromPolicy = false;
+        }
+
+        /**
+         * Initializes the builder with the state of the provided policy. When using this constructor
+         * the identifier field of the builder can not be changed and will result in an IllegalStateException
+         * if attempting to do so.
+         *
+         * @param other the existing access policy to initialize from
+         */
+        public Builder(final AccessPolicy other) {
+            if (other == null) {
+                throw new IllegalArgumentException("Can not initialize builder with a null access policy");
+            }
+
+            this.identifier = other.getIdentifier();
+            this.resource = other.getResource();
+            this.action = other.getAction();
+            this.users.clear();
+            this.users.addAll(other.getUsers());
+            this.groups.clear();
+            this.groups.addAll(other.getGroups());
+            this.fromPolicy = true;
+        }
+
+        /**
+         * Sets the identifier of the builder.
+         *
+         * @param identifier the identifier to set
+         * @return the builder
+         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Policy
+         */
+        public Builder identifier(final String identifier) {
+            if (fromPolicy) {
+                throw new IllegalStateException(
+                        "Identifier can not be changed when initialized from an existing policy");
+            }
+
+            this.identifier = identifier;
+            return this;
+        }
+
+        /**
+         * Sets the identifier of the builder to a random UUID.
+         *
+         * @return the builder
+         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Policy
+         */
+        public Builder identifierGenerateRandom() {
+            if (fromPolicy) {
+                throw new IllegalStateException(
+                        "Identifier can not be changed when initialized from an existing policy");
+            }
+
+            this.identifier = UUID.randomUUID().toString();
+            return this;
+        }
+
+        /**
+         * Sets the identifier of the builder with a UUID generated from the specified seed string.
+         *
+         * @return the builder
+         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Policy
+         */
+        public Builder identifierGenerateFromSeed(final String seed) {
+            if (fromPolicy) {
+                throw new IllegalStateException(
+                        "Identifier can not be changed when initialized from an existing policy");
+            }
+            if (seed == null) {
+                throw new IllegalArgumentException("Cannot seed the policy identifier with a null value.");
+            }
+
+            this.identifier = UUID.nameUUIDFromBytes(seed.getBytes(StandardCharsets.UTF_8)).toString();
+            return this;
+        }
+
+        /**
+         * Sets the resource of the builder.
+         *
+         * @param resource the resource to set
+         * @return the builder
+         */
+        public Builder resource(final String resource) {
+            this.resource = resource;
+            return this;
+        }
+
+        /**
+         * Adds all the users from the provided set to the builder's set of users.
+         *
+         * @param users the users to add
+         * @return the builder
+         */
+        public Builder addUsers(final Set<String> users) {
+            if (users != null) {
+                this.users.addAll(users);
+            }
+            return this;
+        }
+
+        /**
+         * Adds the given user to the builder's set of users.
+         *
+         * @param user the user to add
+         * @return the builder
+         */
+        public Builder addUser(final String user) {
+            if (user != null) {
+                this.users.add(user);
+            }
+            return this;
+        }
+
+        /**
+         * Removes all users in the provided set from the builder's set of users.
+         *
+         * @param users the users to remove
+         * @return the builder
+         */
+        public Builder removeUsers(final Set<String> users) {
+            if (users != null) {
+                this.users.removeAll(users);
+            }
+            return this;
+        }
+
+        /**
+         * Removes the provided user from the builder's set of users.
+         *
+         * @param user the user to remove
+         * @return the builder
+         */
+        public Builder removeUser(final String user) {
+            if (user != null) {
+                this.users.remove(user);
+            }
+            return this;
+        }
+
+        /**
+         * Clears the builder's set of users so that it is non-null and size == 0.
+         *
+         * @return the builder
+         */
+        public Builder clearUsers() {
+            this.users.clear();
+            return this;
+        }
+
+        /**
+         * Adds all the groups from the provided set to the builder's set of groups.
+         *
+         * @param groups the groups to add
+         * @return the builder
+         */
+        public Builder addGroups(final Set<String> groups) {
+            if (groups != null) {
+                this.groups.addAll(groups);
+            }
+            return this;
+        }
+
+        /**
+         * Adds the given group to the builder's set of groups.
+         *
+         * @param group the group to add
+         * @return the builder
+         */
+        public Builder addGroup(final String group) {
+            if (group != null) {
+                this.groups.add(group);
+            }
+            return this;
+        }
+
+        /**
+         * Removes all groups in the provided set from the builder's set of groups.
+         *
+         * @param groups the groups to remove
+         * @return the builder
+         */
+        public Builder removeGroups(final Set<String> groups) {
+            if (groups != null) {
+                this.groups.removeAll(groups);
+            }
+            return this;
+        }
+
+        /**
+         * Removes the provided groups from the builder's set of groups.
+         *
+         * @param group the group to remove
+         * @return the builder
+         */
+        public Builder removeGroup(final String group) {
+            if (group != null) {
+                this.groups.remove(group);
+            }
+            return this;
+        }
+
+        /**
+         * Clears the builder's set of groups so that it is non-null and size == 0.
+         *
+         * @return the builder
+         */
+        public Builder clearGroups() {
+            this.groups.clear();
+            return this;
+        }
+
+        /**
+         * Sets the action for this builder.
+         *
+         * @param action the action to set
+         * @return the builder
+         */
+        public Builder action(final RequestAction action) {
+            this.action = action;
+            return this;
+        }
+
+        /**
+         * @return a new AccessPolicy constructed from the state of the builder
+         */
+        public AccessPolicy build() {
+            return new AccessPolicy(this);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicyProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicyProvider.java
new file mode 100644
index 0000000..b942ec2
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicyProvider.java
@@ -0,0 +1,90 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
+
+import java.util.Set;
+
+/**
+ * Provides access to AccessPolicies and the configured UserGroupProvider.
+ *
+ * NOTE: Extensions will be called often and frequently. Because of this, if the underlying implementation needs to
+ * make remote calls or expensive calculations those should probably be done asynchronously and/or cache the results.
+ *
+ * Additionally, extensions need to be thread safe.
+ */
+public interface AccessPolicyProvider {
+
+    /**
+     * Retrieves all access policies. Must be non null
+     *
+     * @return a list of policies
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    Set<AccessPolicy> getAccessPolicies() throws AuthorizationAccessException;
+
+    /**
+     * Retrieves the policy with the given identifier.
+     *
+     * @param identifier the id of the policy to retrieve
+     * @return the policy with the given id, or null if no matching policy exists
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    AccessPolicy getAccessPolicy(String identifier) throws AuthorizationAccessException;
+
+    /**
+     * Gets the access policies for the specified resource identifier and request action.
+     *
+     * @param resourceIdentifier the resource identifier
+     * @param action the request action
+     * @return the policy matching the resouce and action, or null if no matching policy exists
+     * @throws AuthorizationAccessException if there was any unexpected error performing the operation
+     */
+    AccessPolicy getAccessPolicy(String resourceIdentifier, RequestAction action) throws AuthorizationAccessException;
+
+    /**
+     * Returns the UserGroupProvider for this managed Authorizer. Must be non null
+     *
+     * @return the UserGroupProvider
+     */
+    UserGroupProvider getUserGroupProvider();
+
+    /**
+     * Called immediately after instance creation for implementers to perform additional setup
+     *
+     * @param initializationContext in which to initialize
+     */
+    void initialize(AccessPolicyProviderInitializationContext initializationContext) throws AuthorizerCreationException;
+
+    /**
+     * Called to configure the Authorizer.
+     *
+     * @param configurationContext at the time of configuration
+     * @throws AuthorizerCreationException for any issues configuring the provider
+     */
+    void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException;
+
+    /**
+     * Called immediately before instance destruction for implementers to release resources.
+     *
+     * @throws AuthorizerDestructionException If pre-destruction fails.
+     */
+    void preDestruction() throws AuthorizerDestructionException;
+}


[05/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicyProviderInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicyProviderInitializationContext.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicyProviderInitializationContext.java
new file mode 100644
index 0000000..92792e9
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AccessPolicyProviderInitializationContext.java
@@ -0,0 +1,30 @@
+/*
+ * 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.authorization;
+
+/**
+ * Initialization content for AccessPolicyProviders.
+ */
+public interface AccessPolicyProviderInitializationContext extends UserGroupProviderInitializationContext {
+
+    /**
+     * The lookup for accessing other configured AccessPolicyProviders.
+     *
+     * @return  The AccessPolicyProvider lookup
+     */
+    AccessPolicyProviderLookup getAccessPolicyProviderLookup();
+}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationAuditor.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationAuditor.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationAuditor.java
new file mode 100644
index 0000000..ae01ece
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationAuditor.java
@@ -0,0 +1,30 @@
+/*
+ * 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.authorization;
+
+public interface AuthorizationAuditor {
+
+    /**
+     * Audits an authorization request. Will be invoked for any Approved or Denied results. ResourceNotFound
+     * will either re-attempt authorization using a parent resource or will generate a failure result and
+     * audit that.
+     *
+     * @param request the request for authorization
+     * @param result the authorization result
+     */
+    void auditAccessAttempt(final AuthorizationRequest request, final AuthorizationResult result);
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationRequest.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationRequest.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationRequest.java
new file mode 100644
index 0000000..1eb99f9
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationRequest.java
@@ -0,0 +1,245 @@
+/*
+ * 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.authorization;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Supplier;
+
+/**
+ * Represents an authorization request for a given user/entity performing an action against a resource within some userContext.
+ */
+public class AuthorizationRequest {
+
+    public static final String DEFAULT_EXPLANATION = "Unable to perform the desired action.";
+
+    private final Resource resource;
+    private final Resource requestedResource;
+    private final String identity;
+    private final Set<String> groups;
+    private final RequestAction action;
+    private final boolean isAccessAttempt;
+    private final boolean isAnonymous;
+    private final Map<String, String> userContext;
+    private final Map<String, String> resourceContext;
+    private final Supplier<String> explanationSupplier;
+
+    private AuthorizationRequest(final Builder builder) {
+        Objects.requireNonNull(builder.resource, "The resource is required when creating an authorization request");
+        Objects.requireNonNull(builder.action, "The action is required when creating an authorization request");
+        Objects.requireNonNull(builder.isAccessAttempt, "Whether this request is an access attempt is request");
+        Objects.requireNonNull(builder.isAnonymous, "Whether this request is being performed by an anonymous user is required");
+
+        this.resource = builder.resource;
+        this.identity = builder.identity;
+        this.groups = builder.groups == null ? null : Collections.unmodifiableSet(builder.groups);
+        this.action = builder.action;
+        this.isAccessAttempt = builder.isAccessAttempt;
+        this.isAnonymous = builder.isAnonymous;
+        this.userContext = builder.userContext == null ? null : Collections.unmodifiableMap(builder.userContext);
+        this.resourceContext = builder.resourceContext == null ? null : Collections.unmodifiableMap(builder.resourceContext);
+        this.explanationSupplier = () -> {
+            final String explanation = builder.explanationSupplier.get();
+
+            // ensure the specified supplier returns non null
+            if (explanation == null) {
+                return DEFAULT_EXPLANATION;
+            } else {
+                return explanation;
+            }
+        };
+
+        if (builder.requestedResource == null) {
+            this.requestedResource = builder.resource;
+        } else {
+            this.requestedResource = builder.requestedResource;
+        }
+    }
+
+    /**
+     * The Resource being authorized. Not null.
+     *
+     * @return The resource
+     */
+    public Resource getResource() {
+        return resource;
+    }
+
+    /**
+     * The original Resource being requested. In cases with inherited policies, this will be a ancestor resource of
+     * of the current resource. The initial request, and cases without inheritance, the requested resource will be
+     * the same as the current resource.
+     *
+     * @return The requested resource
+     */
+    public Resource getRequestedResource() {
+        return requestedResource;
+    }
+
+    /**
+     * The identity accessing the Resource. May be null if the user could not authenticate.
+     *
+     * @return The identity
+     */
+    public String getIdentity() {
+        return identity;
+    }
+
+    /**
+     * The groups the user making this request belongs to. May be null if this NiFi is not configured to load user
+     * groups or empty if the user has no groups
+     *
+     * @return The groups
+     */
+    public Set<String> getGroups() {
+        return groups;
+    }
+
+    /**
+     * Whether this is a direct access attempt of the Resource if if it's being checked as part of another response.
+     *
+     * @return if this is a direct access attempt
+     */
+    public boolean isAccessAttempt() {
+        return isAccessAttempt;
+    }
+
+    /**
+     * Whether the entity accessing is anonymous.
+     *
+     * @return whether the entity is anonymous
+     */
+    public boolean isAnonymous() {
+        return isAnonymous;
+    }
+
+    /**
+     * The action being taken against the Resource. Not null.
+     *
+     * @return The action
+     */
+    public RequestAction getAction() {
+        return action;
+    }
+
+    /**
+     * The userContext of the user request to make additional access decisions. May be null.
+     *
+     * @return  The userContext of the user request
+     */
+    public Map<String, String> getUserContext() {
+        return userContext;
+    }
+
+    /**
+     * The event attributes to make additional access decisions for provenance events. May be null.
+     *
+     * @return  The event attributes
+     */
+    public Map<String, String> getResourceContext() {
+        return resourceContext;
+    }
+
+    /**
+     * A supplier for the explanation if access is denied. Non null.
+     *
+     * @return The explanation supplier if access is denied
+     */
+    public Supplier<String> getExplanationSupplier() {
+        return explanationSupplier;
+    }
+
+    /**
+     * AuthorizationRequest builder.
+     */
+    public static final class Builder {
+
+        private Resource resource;
+        private Resource requestedResource;
+        private String identity;
+        private Set<String> groups;
+        private Boolean isAnonymous;
+        private Boolean isAccessAttempt;
+        private RequestAction action;
+        private Map<String, String> userContext;
+        private Map<String, String> resourceContext;
+        private Supplier<String> explanationSupplier = () -> DEFAULT_EXPLANATION;
+
+        public Builder resource(final Resource resource) {
+            this.resource = resource;
+            return this;
+        }
+
+        public Builder requestedResource(final Resource requestedResource) {
+            this.requestedResource = requestedResource;
+            return this;
+        }
+
+        public Builder identity(final String identity) {
+            this.identity = identity;
+            return this;
+        }
+
+        public Builder groups(final Set<String> groups) {
+            this.groups = groups;
+            return this;
+        }
+
+        public Builder anonymous(final Boolean isAnonymous) {
+            this.isAnonymous = isAnonymous;
+            return this;
+        }
+
+        public Builder accessAttempt(final Boolean isAccessAttempt) {
+            this.isAccessAttempt = isAccessAttempt;
+            return this;
+        }
+
+        public Builder action(final RequestAction action) {
+            this.action = action;
+            return this;
+        }
+
+        public Builder userContext(final Map<String, String> userContext) {
+            if (userContext != null) {
+                this.userContext = new HashMap<>(userContext);
+            }
+            return this;
+        }
+
+        public Builder resourceContext(final Map<String, String> resourceContext) {
+            if (resourceContext != null) {
+                this.resourceContext = new HashMap<>(resourceContext);
+            }
+            return this;
+        }
+
+        public Builder explanationSupplier(final Supplier<String> explanationSupplier) {
+            if (explanationSupplier != null) {
+                this.explanationSupplier = explanationSupplier;
+            }
+            return this;
+        }
+
+        public AuthorizationRequest build() {
+            return new AuthorizationRequest(this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationResult.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationResult.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationResult.java
new file mode 100644
index 0000000..5f9b55e
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizationResult.java
@@ -0,0 +1,103 @@
+/*
+ * 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.authorization;
+
+/**
+ * Represents a decision whether authorization is granted.
+ */
+public class AuthorizationResult {
+
+    public enum Result {
+        Approved,
+        Denied,
+        ResourceNotFound
+    }
+
+    private static final AuthorizationResult APPROVED = new AuthorizationResult(Result.Approved, null);
+    private static final AuthorizationResult RESOURCE_NOT_FOUND = new AuthorizationResult(Result.ResourceNotFound, "Not authorized for the requested resource.");
+
+    private final Result result;
+    private final String explanation;
+
+    /**
+     * Creates a new AuthorizationResult with the specified result and explanation.
+     *
+     * @param result of the authorization
+     * @param explanation for the authorization attempt
+     */
+    private AuthorizationResult(Result result, String explanation) {
+        if (Result.Denied.equals(result) && explanation == null) {
+            throw new IllegalArgumentException("An explanation is required when the authorization request is denied.");
+        }
+
+        if (Result.ResourceNotFound.equals(result) && explanation == null) {
+            throw new IllegalArgumentException("An explanation is required when the authorization request is resource not found.");
+        }
+
+        this.result = result;
+        this.explanation = explanation;
+    }
+
+    /**
+     * @return Whether or not the request is approved
+     */
+    public Result getResult() {
+        return result;
+    }
+
+    /**
+     * @return If the request is denied, the reason why. Null otherwise
+     */
+    public String getExplanation() {
+        return explanation;
+    }
+
+    /**
+     * @return a new approved AuthorizationResult
+     */
+    public static AuthorizationResult approved() {
+        return APPROVED;
+    }
+
+    /**
+     * Resource not found will indicate that there are no specific authorization rules for this resource.
+     * @return a new resource not found AuthorizationResult
+     */
+    public static AuthorizationResult resourceNotFound() {
+        return RESOURCE_NOT_FOUND;
+    }
+
+    /**
+     * Creates a new denied AuthorizationResult with a message indicating 'Access is denied'.
+     *
+     * @return a new denied AuthorizationResult
+     */
+    public static AuthorizationResult denied() {
+        return denied(AuthorizationRequest.DEFAULT_EXPLANATION);
+    }
+
+    /**
+     * Creates a new denied AuthorizationResult with the specified explanation.
+     *
+     * @param explanation for why it was denied
+     * @return a new denied AuthorizationResult with the specified explanation
+     * @throws IllegalArgumentException if explanation is null
+     */
+    public static AuthorizationResult denied(String explanation) {
+        return new AuthorizationResult(Result.Denied, explanation);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Authorizer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Authorizer.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Authorizer.java
new file mode 100644
index 0000000..3018dd3
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Authorizer.java
@@ -0,0 +1,63 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
+
+/**
+ * Authorizes user requests.
+ */
+public interface Authorizer {
+
+    /**
+     * Determines if the specified user/entity is authorized to access the specified resource within the given context.
+     * These details are all contained in the AuthorizationRequest.
+     *
+     * NOTE: This method will be called often and frequently. Because of this, if the underlying implementation needs to
+     * make remote calls or expensive calculations those should probably be done asynchronously and/or cache the results.
+     *
+     * @param   request The authorization request
+     * @return  the authorization result
+     * @throws  AuthorizationAccessException if unable to access the policies
+     */
+    AuthorizationResult authorize(AuthorizationRequest request) throws AuthorizationAccessException;
+
+    /**
+     * Called immediately after instance creation for implementers to perform additional setup
+     *
+     * @param initializationContext in which to initialize
+     */
+    void initialize(AuthorizerInitializationContext initializationContext) throws AuthorizerCreationException;
+
+    /**
+     * Called to configure the Authorizer.
+     *
+     * @param configurationContext at the time of configuration
+     * @throws AuthorizerCreationException for any issues configuring the provider
+     */
+    void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException;
+
+    /**
+     * Called immediately before instance destruction for implementers to release resources.
+     *
+     * @throws AuthorizerDestructionException If pre-destruction fails.
+     */
+    void preDestruction() throws AuthorizerDestructionException;
+
+}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerInitializationContext.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerInitializationContext.java
new file mode 100644
index 0000000..55854b3
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/AuthorizerInitializationContext.java
@@ -0,0 +1,30 @@
+/*
+ * 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.authorization;
+
+/**
+ * Initialization content for Authorizers.
+ */
+public interface AuthorizerInitializationContext extends AccessPolicyProviderInitializationContext {
+
+    /**
+     * The lookup for accessing other configured Authorizers.
+     *
+     * @return  The Authorizer lookup
+     */
+    AuthorizerLookup getAuthorizerLookup();
+}

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ConfigurableAccessPolicyProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ConfigurableAccessPolicyProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ConfigurableAccessPolicyProvider.java
new file mode 100644
index 0000000..1f909a4
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ConfigurableAccessPolicyProvider.java
@@ -0,0 +1,108 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.UninheritableAuthorizationsException;
+
+/**
+ * Provides support for configuring AccessPolicies.
+ *
+ * NOTE: Extensions will be called often and frequently. Because of this, if the underlying implementation needs to
+ * make remote calls or expensive calculations those should probably be done asynchronously and/or cache the results.
+ *
+ * Additionally, extensions need to be thread safe.
+ */
+public interface ConfigurableAccessPolicyProvider extends AccessPolicyProvider {
+
+    /**
+     * Returns a fingerprint representing the authorizations managed by this authorizer. The fingerprint will be
+     * used for comparison to determine if two policy-based authorizers represent a compatible set of policies.
+     *
+     * @return the fingerprint for this Authorizer
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    String getFingerprint() throws AuthorizationAccessException;
+
+    /**
+     * Parses the fingerprint and adds any policies to the current AccessPolicyProvider.
+     *
+     * @param fingerprint the fingerprint that was obtained from calling getFingerprint() on another Authorizer.
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    void inheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
+
+    /**
+     * When the fingerprints are not equal, this method will check if the proposed fingerprint is inheritable.
+     * If the fingerprint is an exact match, this method will not be invoked as there is nothing to inherit.
+     *
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws UninheritableAuthorizationsException if the proposed fingerprint was uninheritable
+     */
+    void checkInheritability(final String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException;
+
+    /**
+     * Adds the given policy ensuring that multiple policies can not be added for the same resource and action.
+     *
+     * @param accessPolicy the policy to add
+     * @return the policy that was added
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    AccessPolicy addAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException;
+
+    /**
+     * Determines whether the specified access policy is configurable. Provides the opportunity for a ConfigurableAccessPolicyProvider to prevent
+     * editing of a specific access policy. By default, all known access policies are configurable.
+     *
+     * @param accessPolicy the access policy
+     * @return is configurable
+     */
+    default boolean isConfigurable(AccessPolicy accessPolicy) {
+        if (accessPolicy == null) {
+            throw new IllegalArgumentException("Access policy cannot be null");
+        }
+
+        return getAccessPolicy(accessPolicy.getIdentifier()) != null;
+    }
+
+    /**
+     * The policy represented by the provided instance will be updated based on the provided instance.
+     *
+     * @param accessPolicy an updated policy
+     * @return the updated policy, or null if no matching policy was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    AccessPolicy updateAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the given policy.
+     *
+     * @param accessPolicy the policy to delete
+     * @return the deleted policy, or null if no matching policy was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    AccessPolicy deleteAccessPolicy(AccessPolicy accessPolicy) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the policy with the specified identifier.
+     *
+     * @param accessPolicyIdentifier the policy to delete
+     * @return the deleted policy, or null if no matching policy was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    AccessPolicy deleteAccessPolicy(String accessPolicyIdentifier) throws AuthorizationAccessException;
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ConfigurableUserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ConfigurableUserGroupProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ConfigurableUserGroupProvider.java
new file mode 100644
index 0000000..bd52128
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ConfigurableUserGroupProvider.java
@@ -0,0 +1,163 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.UninheritableAuthorizationsException;
+
+/**
+ * Provides support for configuring Users and Groups.
+ *
+ * NOTE: Extensions will be called often and frequently. Because of this, if the underlying implementation needs to
+ * make remote calls or expensive calculations those should probably be done asynchronously and/or cache the results.
+ *
+ * Additionally, extensions need to be thread safe.
+ */
+public interface ConfigurableUserGroupProvider extends UserGroupProvider {
+
+    /**
+     * Returns a fingerprint representing the authorizations managed by this authorizer. The fingerprint will be
+     * used for comparison to determine if two policy-based authorizers represent a compatible set of users and/or groups.
+     *
+     * @return the fingerprint for this Authorizer
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    String getFingerprint() throws AuthorizationAccessException;
+
+    /**
+     * Parses the fingerprint and adds any users and groups to the current Authorizer.
+     *
+     * @param fingerprint the fingerprint that was obtained from calling getFingerprint() on another Authorizer.
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    void inheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
+
+    /**
+     * When the fingerprints are not equal, this method will check if the proposed fingerprint is inheritable.
+     * If the fingerprint is an exact match, this method will not be invoked as there is nothing to inherit.
+     *
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws UninheritableAuthorizationsException if the proposed fingerprint was uninheritable
+     */
+    void checkInheritability(final String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException;
+
+    /**
+     * Adds the given user.
+     *
+     * @param user the user to add
+     * @return the user that was added
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws IllegalStateException if there is already a user with the same identity
+     */
+    User addUser(User user) throws AuthorizationAccessException;
+
+    /**
+     * Determines whether the specified user is configurable. Provides the opportunity for a ConfigurableUserGroupProvider to prevent
+     * editing of a specific user. By default, all known users are configurable.
+     *
+     * @param user the user
+     * @return is configurable
+     */
+    default boolean isConfigurable(User user) {
+        if (user == null) {
+            throw new IllegalArgumentException("User cannot be null");
+        }
+
+        return getUser(user.getIdentifier()) != null;
+    }
+
+    /**
+     * The user represented by the provided instance will be updated based on the provided instance.
+     *
+     * @param user an updated user instance
+     * @return the updated user instance, or null if no matching user was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws IllegalStateException if there is already a user with the same identity
+     */
+    User updateUser(final User user) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the given user.
+     *
+     * @param user the user to delete
+     * @return the user that was deleted, or null if no matching user was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    User deleteUser(User user) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the user for the given ID.
+     *
+     * @param userIdentifier the user to delete
+     * @return the user that was deleted, or null if no matching user was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    User deleteUser(String userIdentifier) throws AuthorizationAccessException;
+
+    /**
+     * Adds a new group.
+     *
+     * @param group the Group to add
+     * @return the added Group
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws IllegalStateException if a group with the same name already exists
+     */
+    Group addGroup(Group group) throws AuthorizationAccessException;
+
+    /**
+     * Determines whether the specified group is configurable. Provides the opportunity for a ConfigurableUserGroupProvider to prevent
+     * editing of a specific group. By default, all known groups are configurable.
+     *
+     * @param group the group
+     * @return is configurable
+     */
+    default boolean isConfigurable(Group group) {
+        if (group == null) {
+            throw new IllegalArgumentException("Group cannot be null");
+        }
+
+        return getGroup(group.getIdentifier()) != null;
+    }
+
+    /**
+     * The group represented by the provided instance will be updated based on the provided instance.
+     *
+     * @param group an updated group instance
+     * @return the updated group instance, or null if no matching group was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws IllegalStateException if there is already a group with the same name
+     */
+    Group updateGroup(Group group) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the given group.
+     *
+     * @param group the group to delete
+     * @return the deleted group, or null if no matching group was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    Group deleteGroup(Group group) throws AuthorizationAccessException;
+
+    /**
+     * Deletes the given group.
+     *
+     * @param groupIdentifier the group to delete
+     * @return the deleted group, or null if no matching group was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    Group deleteGroup(String groupIdentifier) throws AuthorizationAccessException;
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Group.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Group.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Group.java
new file mode 100644
index 0000000..0ec7764
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Group.java
@@ -0,0 +1,263 @@
+/*
+ * 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.authorization;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+
+/**
+ * A group that users can belong to.
+ */
+public class Group {
+
+    private final String identifier;
+
+    private final String name;
+
+    private final Set<String> users;
+
+    private Group(final Builder builder) {
+        this.identifier = builder.identifier;
+        this.name = builder.name;
+        this.users = Collections.unmodifiableSet(new HashSet<>(builder.users));
+
+        if (this.identifier == null || this.identifier.trim().isEmpty()) {
+            throw new IllegalArgumentException("Identifier can not be null or empty");
+        }
+
+        if (this.name == null || this.name.trim().isEmpty()) {
+            throw new IllegalArgumentException("Name can not be null or empty");
+        }
+    }
+
+    /**
+     * @return the identifier of the group
+     */
+    public String getIdentifier() {
+        return identifier;
+    }
+
+    /**
+     * @return the name of the group
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * @return an unmodifiable set of user identifiers that belong to this group
+     */
+    public Set<String> getUsers() {
+        return users;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+
+        final Group other = (Group) obj;
+        return Objects.equals(this.identifier, other.identifier);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(this.identifier);
+    }
+
+    @Override
+    public String toString() {
+        return String.format("identifier[%s], name[%s]", getIdentifier(), getName());
+    }
+
+
+    /**
+     * Builder for creating Groups.
+     */
+    public static class Builder {
+
+        private String identifier;
+        private String name;
+        private Set<String> users = new HashSet<>();
+        private final boolean fromGroup;
+
+        public Builder() {
+            this.fromGroup = false;
+        }
+
+        /**
+         * Initializes the builder with the state of the provided group. When using this constructor
+         * the identifier field of the builder can not be changed and will result in an IllegalStateException
+         * if attempting to do so.
+         *
+         * @param other the existing access policy to initialize from
+         */
+        public Builder(final Group other) {
+            if (other == null) {
+                throw new IllegalArgumentException("Provided group can not be null");
+            }
+
+            this.identifier = other.getIdentifier();
+            this.name = other.getName();
+            this.users.clear();
+            this.users.addAll(other.getUsers());
+            this.fromGroup = true;
+        }
+
+        /**
+         * Sets the identifier of the builder.
+         *
+         * @param identifier the identifier
+         * @return the builder
+         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Group
+         */
+        public Builder identifier(final String identifier) {
+            if (fromGroup) {
+                throw new IllegalStateException(
+                        "Identifier can not be changed when initialized from an existing group");
+            }
+
+            this.identifier = identifier;
+            return this;
+        }
+
+        /**
+         * Sets the identifier of the builder to a random UUID.
+         *
+         * @return the builder
+         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Group
+         */
+        public Builder identifierGenerateRandom() {
+            if (fromGroup) {
+                throw new IllegalStateException(
+                        "Identifier can not be changed when initialized from an existing group");
+            }
+
+            this.identifier = UUID.randomUUID().toString();
+            return this;
+        }
+
+        /**
+         * Sets the identifier of the builder with a UUID generated from the specified seed string.
+         *
+         * @return the builder
+         * @throws IllegalStateException if this method is called when this builder was constructed from an existing Group
+         */
+        public Builder identifierGenerateFromSeed(final String seed) {
+            if (fromGroup) {
+                throw new IllegalStateException(
+                        "Identifier can not be changed when initialized from an existing group");
+            }
+            if (seed == null) {
+                throw new IllegalArgumentException("Cannot seed the group identifier with a null value.");
+            }
+
+            this.identifier = UUID.nameUUIDFromBytes(seed.getBytes(StandardCharsets.UTF_8)).toString();
+            return this;
+        }
+
+        /**
+         * Sets the name of the builder.
+         *
+         * @param name the name
+         * @return the builder
+         */
+        public Builder name(final String name) {
+            this.name = name;
+            return this;
+        }
+
+        /**
+         * Adds all users from the provided set to the builder's set of users.
+         *
+         * @param users a set of users to add
+         * @return the builder
+         */
+        public Builder addUsers(final Set<String> users) {
+            if (users != null) {
+                this.users.addAll(users);
+            }
+            return this;
+        }
+
+        /**
+         * Adds the given user to the builder's set of users.
+         *
+         * @param user the user to add
+         * @return the builder
+         */
+        public Builder addUser(final String user) {
+            if (user != null) {
+                this.users.add(user);
+            }
+            return this;
+        }
+
+        /**
+         * Removes the given user from the builder's set of users.
+         *
+         * @param user the user to remove
+         * @return the builder
+         */
+        public Builder removeUser(final String user) {
+            if (user != null) {
+                this.users.remove(user);
+            }
+            return this;
+        }
+
+        /**
+         * Removes all users from the provided set from the builder's set of users.
+         *
+         * @param users the users to remove
+         * @return the builder
+         */
+        public Builder removeUsers(final Set<String> users) {
+            if (users != null) {
+                this.users.removeAll(users);
+            }
+            return this;
+        }
+
+        /**
+         * Clears the builder's set of users so that users is non-null with size 0.
+         *
+         * @return the builder
+         */
+        public Builder clearUsers() {
+            this.users.clear();
+            return this;
+        }
+
+        /**
+         * @return a new Group constructed from the state of the builder
+         */
+        public Group build() {
+            return new Group(this);
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ManagedAuthorizer.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ManagedAuthorizer.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ManagedAuthorizer.java
new file mode 100644
index 0000000..50b8094
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/ManagedAuthorizer.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.security.authorization;
+
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.UninheritableAuthorizationsException;
+
+public interface ManagedAuthorizer extends Authorizer {
+
+    /**
+     * Returns a fingerprint representing the authorizations managed by this authorizer. The fingerprint will be
+     * used for comparison to determine if two managed authorizers represent a compatible set of users,
+     * groups, and/or policies. Must be non null
+     *
+     * @return the fingerprint for this Authorizer
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    String getFingerprint() throws AuthorizationAccessException;
+
+    /**
+     * Parses the fingerprint and adds any users, groups, and policies to the current Authorizer.
+     *
+     * @param fingerprint the fingerprint that was obtained from calling getFingerprint() on another Authorizer.
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    void inheritFingerprint(final String fingerprint) throws AuthorizationAccessException;
+
+    /**
+     * When the fingerprints are not equal, this method will check if the proposed fingerprint is inheritable.
+     * If the fingerprint is an exact match, this method will not be invoked as there is nothing to inherit.
+     *
+     * @param proposedFingerprint the proposed fingerprint
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     * @throws UninheritableAuthorizationsException if the proposed fingerprint was uninheritable
+     */
+    void checkInheritability(final String proposedFingerprint) throws AuthorizationAccessException, UninheritableAuthorizationsException;
+
+    /**
+     * Returns the AccessPolicy provider for this managed Authorizer. Must be non null
+     *
+     * @return the AccessPolicy provider
+     */
+    AccessPolicyProvider getAccessPolicyProvider();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/RequestAction.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/RequestAction.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/RequestAction.java
new file mode 100644
index 0000000..adf07f2
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/RequestAction.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.security.authorization;
+
+import java.util.StringJoiner;
+
+/**
+ * Actions a user/entity can take on a resource.
+ */
+public enum RequestAction {
+    READ("read"),
+    WRITE("write"),
+    DELETE("delete");
+
+    private String value;
+
+    RequestAction(String value) {
+        this.value = value;
+    }
+
+    @Override
+    public String toString() {
+        return value.toLowerCase();
+    }
+
+    public static RequestAction valueOfValue(final String action) {
+        if (RequestAction.READ.toString().equalsIgnoreCase(action)) {
+            return RequestAction.READ;
+        } else if (RequestAction.WRITE.toString().equalsIgnoreCase(action)) {
+            return RequestAction.WRITE;
+        } else if (RequestAction.DELETE.toString().equalsIgnoreCase(action)) {
+            return RequestAction.DELETE;
+        } else {
+            StringJoiner stringJoiner = new StringJoiner(", ");
+            for(RequestAction ra : RequestAction.values()) {
+                stringJoiner.add(ra.toString());
+            }
+            String allowableValues = stringJoiner.toString();
+            throw new IllegalArgumentException("Action must be one of [" + allowableValues + "]");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Resource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Resource.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Resource.java
new file mode 100644
index 0000000..eacdffe
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/Resource.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.security.authorization;
+
+/**
+ * Resource in an authorization request.
+ */
+public interface Resource {
+
+    /**
+     * The identifier for this resource.
+     *
+     * @return identifier for this resource
+     */
+    String getIdentifier();
+
+    /**
+     * The name of this resource. May be null.
+     *
+     * @return name of this resource
+     */
+    String getName();
+
+    /**
+     * The description of this resource that may be safely used in messages to the client.
+     *
+     * @return safe description
+     */
+    String getSafeDescription();
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/User.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/User.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/User.java
new file mode 100644
index 0000000..8879afe
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/User.java
@@ -0,0 +1,188 @@
+/*
+ * 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.authorization;
+
+import java.nio.charset.StandardCharsets;
+import java.util.Objects;
+import java.util.UUID;
+
+/**
+ * A user to create authorization policies for.
+ */
+public class User {
+
+    private final String identifier;
+
+    private final String identity;
+
+    private User(final Builder builder) {
+        this.identifier = builder.identifier;
+        this.identity = builder.identity;
+
+        if (identifier == null || identifier.trim().isEmpty()) {
+            throw new IllegalArgumentException("Identifier can not be null or empty");
+        }
+
+        if (identity == null || identity.trim().isEmpty()) {
+            throw new IllegalArgumentException("Identity can not be null or empty");
+        }
+
+    }
+
+    /**
+     * @return the identifier of the user
+     */
+    public String getIdentifier() {
+        return identifier;
+    }
+
+    /**
+     * @return the identity string of the user
+     */
+    public String getIdentity() {
+        return identity;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == null) {
+            return false;
+        }
+        if (getClass() != obj.getClass()) {
+            return false;
+        }
+
+        final User other = (User) obj;
+        return Objects.equals(this.identifier, other.identifier);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(this.identifier);
+    }
+
+    @Override
+    public String toString() {
+        return String.format("identifier[%s], identity[%s]", getIdentifier(), getIdentity());
+    }
+
+    /**
+     * Builder for Users.
+     */
+    public static class Builder {
+
+        private String identifier;
+        private String identity;
+        private final boolean fromUser;
+
+        /**
+         * Default constructor for building a new User.
+         */
+        public Builder() {
+            this.fromUser = false;
+        }
+
+        /**
+         * Initializes the builder with the state of the provided user. When using this constructor
+         * the identifier field of the builder can not be changed and will result in an IllegalStateException
+         * if attempting to do so.
+         *
+         * @param other the existing user to initialize from
+         */
+        public Builder(final User other) {
+            if (other == null) {
+                throw new IllegalArgumentException("Provided user can not be null");
+            }
+
+            this.identifier = other.getIdentifier();
+            this.identity = other.getIdentity();
+            this.fromUser = true;
+        }
+
+        /**
+         * Sets the identifier of the builder.
+         *
+         * @param identifier the identifier to set
+         * @return the builder
+         * @throws IllegalStateException if this method is called when this builder was constructed from an existing User
+         */
+        public Builder identifier(final String identifier) {
+            if (fromUser) {
+                throw new IllegalStateException(
+                        "Identifier can not be changed when initialized from an existing user");
+            }
+
+            this.identifier = identifier;
+            return this;
+        }
+
+        /**
+         * Sets the identifier of the builder to a random UUID.
+         *
+         * @return the builder
+         * @throws IllegalStateException if this method is called when this builder was constructed from an existing User
+         */
+        public Builder identifierGenerateRandom() {
+            if (fromUser) {
+                throw new IllegalStateException(
+                        "Identifier can not be changed when initialized from an existing user");
+            }
+
+            this.identifier = UUID.randomUUID().toString();
+            return this;
+        }
+
+        /**
+         * Sets the identifier of the builder with a UUID generated from the specified seed string.
+         *
+         * @return the builder
+         * @throws IllegalStateException if this method is called when this builder was constructed from an existing User
+         */
+        public Builder identifierGenerateFromSeed(final String seed) {
+            if (fromUser) {
+                throw new IllegalStateException(
+                        "Identifier can not be changed when initialized from an existing user");
+            }
+            if (seed == null) {
+                throw new IllegalArgumentException("Cannot seed the user identifier with a null value.");
+            }
+
+            this.identifier = UUID.nameUUIDFromBytes(seed.getBytes(StandardCharsets.UTF_8)).toString();
+            return this;
+        }
+
+        /**
+         * Sets the identity of the builder.
+         *
+         * @param identity the identity to set
+         * @return the builder
+         */
+        public Builder identity(final String identity) {
+            this.identity = identity;
+            return this;
+        }
+
+        /**
+         * @return a new User constructed from the state of the builder
+         */
+        public User build() {
+            return new User(this);
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserAndGroups.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserAndGroups.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserAndGroups.java
new file mode 100644
index 0000000..c9cd0aa
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserAndGroups.java
@@ -0,0 +1,40 @@
+/*
+ * 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.authorization;
+
+import java.util.Set;
+
+/**
+ * A holder object to provide atomic access to a user and their groups.
+ */
+public interface UserAndGroups {
+
+    /**
+     * Retrieves the user, or null if the user is unknown
+     *
+     * @return the user with the given identity
+     */
+    User getUser();
+
+    /**
+     * Retrieves the groups for the user, or null if the user is unknown or has no groups.
+     *
+     * @return the set of groups for the given user identity
+     */
+    Set<Group> getGroups();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserContextKeys.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserContextKeys.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserContextKeys.java
new file mode 100644
index 0000000..8db6cfc
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserContextKeys.java
@@ -0,0 +1,26 @@
+/*
+ * 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.authorization;
+
+/**
+ * Constants for keys that can be passed in the AuthorizationRequest user context Map.
+ */
+public enum UserContextKeys {
+
+    CLIENT_ADDRESS;
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserGroupProvider.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserGroupProvider.java
new file mode 100644
index 0000000..9811691
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserGroupProvider.java
@@ -0,0 +1,108 @@
+/*
+ * 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.authorization;
+
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
+
+import java.util.Set;
+
+/**
+ * Provides access to Users and Groups.
+ *
+ * NOTE: Extensions will be called often and frequently. Because of this, if the underlying implementation needs to
+ * make remote calls or expensive calculations those should probably be done asynchronously and/or cache the results.
+ *
+ * Additionally, extensions need to be thread safe.
+ */
+public interface UserGroupProvider {
+
+    /**
+     * Retrieves all users. Must be non null
+     *
+     * @return a list of users
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    Set<User> getUsers() throws AuthorizationAccessException;
+
+    /**
+     * Retrieves the user with the given identifier.
+     *
+     * @param identifier the id of the user to retrieve
+     * @return the user with the given id, or null if no matching user was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    User getUser(String identifier) throws AuthorizationAccessException;
+
+    /**
+     * Retrieves the user with the given identity.
+     *
+     * @param identity the identity of the user to retrieve
+     * @return the user with the given identity, or null if no matching user was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    User getUserByIdentity(String identity) throws AuthorizationAccessException;
+
+    /**
+     * Retrieves all groups. Must be non null
+     *
+     * @return a list of groups
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    Set<Group> getGroups() throws AuthorizationAccessException;
+
+    /**
+     * Retrieves a Group by id.
+     *
+     * @param identifier the identifier of the Group to retrieve
+     * @return the Group with the given identifier, or null if no matching group was found
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    Group getGroup(String identifier) throws AuthorizationAccessException;
+
+    /**
+     * Gets a user and their groups. Must be non null. If the user is not known the UserAndGroups.getUser() and
+     * UserAndGroups.getGroups() should return null
+     *
+     * @return the UserAndGroups for the specified identity
+     * @throws AuthorizationAccessException if there was an unexpected error performing the operation
+     */
+    UserAndGroups getUserAndGroups(String identity) throws AuthorizationAccessException;
+
+    /**
+     * Called immediately after instance creation for implementers to perform additional setup
+     *
+     * @param initializationContext in which to initialize
+     */
+    void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException;
+
+    /**
+     * Called to configure the Authorizer.
+     *
+     * @param configurationContext at the time of configuration
+     * @throws AuthorizerCreationException for any issues configuring the provider
+     */
+    void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException;
+
+    /**
+     * Called immediately before instance destruction for implementers to release resources.
+     *
+     * @throws AuthorizerDestructionException If pre-destruction fails.
+     */
+    void preDestruction() throws AuthorizerDestructionException;
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserGroupProviderInitializationContext.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserGroupProviderInitializationContext.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserGroupProviderInitializationContext.java
new file mode 100644
index 0000000..d2c471e
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/UserGroupProviderInitializationContext.java
@@ -0,0 +1,37 @@
+/*
+ * 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.authorization;
+
+/**
+ * Initialization content for UserGroupProviders.
+ */
+public interface UserGroupProviderInitializationContext {
+
+    /**
+     * The identifier of the UserGroupProvider.
+     *
+     * @return  The identifier
+     */
+    String getIdentifier();
+
+    /**
+     * The lookup for accessing other configured UserGroupProviders.
+     *
+     * @return  The UserGroupProvider lookup
+     */
+    UserGroupProviderLookup getUserGroupProviderLookup();
+}

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

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AccessDeniedException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AccessDeniedException.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AccessDeniedException.java
new file mode 100644
index 0000000..6ab629c
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/AccessDeniedException.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.security.authorization.exception;
+
+/**
+ * Represents any error that might occur while authorizing user requests.
+ */
+public class AccessDeniedException extends RuntimeException {
+    private static final long serialVersionUID = -5683444815269084134L;
+
+    public AccessDeniedException(Throwable cause) {
+        super(cause);
+    }
+
+    public AccessDeniedException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public AccessDeniedException(String message) {
+        super(message);
+    }
+
+    public AccessDeniedException() {
+    }
+}

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

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


[04/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

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

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/UninheritableAuthorizationsException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/UninheritableAuthorizationsException.java b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/UninheritableAuthorizationsException.java
new file mode 100644
index 0000000..b3ef068
--- /dev/null
+++ b/nifi-registry-security-api/src/main/java/org/apache/nifi/registry/security/authorization/exception/UninheritableAuthorizationsException.java
@@ -0,0 +1,28 @@
+/*
+ * 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.authorization.exception;
+
+/**
+ * Represents the case when the proposed authorizations are not inheritable.
+ */
+public class UninheritableAuthorizationsException extends RuntimeException {
+
+    public UninheritableAuthorizationsException(String message) {
+        super(message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/CertificateUtils.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/CertificateUtils.java b/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/CertificateUtils.java
index bbc7d7c..b14cb61 100644
--- a/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/CertificateUtils.java
+++ b/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/CertificateUtils.java
@@ -17,6 +17,8 @@
 package org.apache.nifi.registry.security.util;
 
 import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.security.util.KeyStoreUtils;
+import org.apache.nifi.registry.security.util.KeystoreType;
 import org.bouncycastle.asn1.ASN1Encodable;
 import org.bouncycastle.asn1.ASN1ObjectIdentifier;
 import org.bouncycastle.asn1.ASN1Set;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/SslContextFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/SslContextFactory.java b/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/SslContextFactory.java
new file mode 100644
index 0000000..9ed8ace
--- /dev/null
+++ b/nifi-registry-security-utils/src/main/java/org/apache/nifi/registry/security/util/SslContextFactory.java
@@ -0,0 +1,249 @@
+/*
+ * 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.net.ssl.KeyManager;
+import javax.net.ssl.KeyManagerFactory;
+import javax.net.ssl.SSLContext;
+import javax.net.ssl.TrustManager;
+import javax.net.ssl.TrustManagerFactory;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.security.KeyManagementException;
+import java.security.KeyStore;
+import java.security.KeyStoreException;
+import java.security.NoSuchAlgorithmException;
+import java.security.SecureRandom;
+import java.security.UnrecoverableKeyException;
+import java.security.cert.CertificateException;
+
+/**
+ * A factory for creating SSL contexts using the application's security
+ * properties.
+ *
+ */
+public final class SslContextFactory {
+
+    public static enum ClientAuth {
+
+        WANT,
+        REQUIRED,
+        NONE
+    }
+
+    /**
+     * Creates a SSLContext instance using the given information. The password for the key is assumed to be the same
+     * as the password for the keystore. If this is not the case, the {@link #createSslContext(String, char[], chart[], String, String, char[], String, ClientAuth, String)}
+     * method should be used instead
+     *
+     * @param keystore the full path to the keystore
+     * @param keystorePasswd the keystore password
+     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
+     * @param truststore the full path to the truststore
+     * @param truststorePasswd the truststore password
+     * @param truststoreType the type of truststore (e.g., PKCS12, JKS)
+     * @param clientAuth the type of client authentication
+     * @param protocol         the protocol to use for the SSL connection
+     *
+     * @return a SSLContext instance
+     * @throws KeyStoreException if any issues accessing the keystore
+     * @throws IOException for any problems loading the keystores
+     * @throws NoSuchAlgorithmException if an algorithm is found to be used but is unknown
+     * @throws CertificateException if there is an issue with the certificate
+     * @throws UnrecoverableKeyException if the key is insufficient
+     * @throws KeyManagementException if unable to manage the key
+     */
+    public static SSLContext createSslContext(
+            final String keystore, final char[] keystorePasswd, final String keystoreType,
+            final String truststore, final char[] truststorePasswd, final String truststoreType,
+            final ClientAuth clientAuth, final String protocol)
+            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
+            UnrecoverableKeyException, KeyManagementException {
+
+        // Pass the keystore password as both the keystore password and the key password.
+        return createSslContext(keystore, keystorePasswd, keystorePasswd, keystoreType, truststore, truststorePasswd, truststoreType, clientAuth, protocol);
+    }
+
+    /**
+     * Creates a SSLContext instance using the given information.
+     *
+     * @param keystore the full path to the keystore
+     * @param keystorePasswd the keystore password
+     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
+     * @param truststore the full path to the truststore
+     * @param truststorePasswd the truststore password
+     * @param truststoreType the type of truststore (e.g., PKCS12, JKS)
+     * @param clientAuth the type of client authentication
+     * @param protocol         the protocol to use for the SSL connection
+     *
+     * @return a SSLContext instance
+     * @throws KeyStoreException if any issues accessing the keystore
+     * @throws IOException for any problems loading the keystores
+     * @throws NoSuchAlgorithmException if an algorithm is found to be used but is unknown
+     * @throws CertificateException if there is an issue with the certificate
+     * @throws UnrecoverableKeyException if the key is insufficient
+     * @throws KeyManagementException if unable to manage the key
+     */
+    public static SSLContext createSslContext(
+            final String keystore, final char[] keystorePasswd, final char[] keyPasswd, final String keystoreType,
+            final String truststore, final char[] truststorePasswd, final String truststoreType,
+            final ClientAuth clientAuth, final String protocol)
+            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
+            UnrecoverableKeyException, KeyManagementException {
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStoreUtils.getKeyStore(keystoreType);
+        try (final InputStream keyStoreStream = new FileInputStream(keystore)) {
+            keyStore.load(keyStoreStream, keystorePasswd);
+        }
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        if (keyPasswd == null) {
+            keyManagerFactory.init(keyStore, keystorePasswd);
+        } else {
+            keyManagerFactory.init(keyStore, keyPasswd);
+        }
+
+        // prepare the truststore
+        final KeyStore trustStore = KeyStoreUtils.getTrustStore(truststoreType);
+        try (final InputStream trustStoreStream = new FileInputStream(truststore)) {
+            trustStore.load(trustStoreStream, truststorePasswd);
+        }
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+        trustManagerFactory.init(trustStore);
+
+        // initialize the ssl context
+        final SSLContext sslContext = SSLContext.getInstance(protocol);
+        sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), new SecureRandom());
+        if (ClientAuth.REQUIRED == clientAuth) {
+            sslContext.getDefaultSSLParameters().setNeedClientAuth(true);
+        } else if (ClientAuth.WANT == clientAuth) {
+            sslContext.getDefaultSSLParameters().setWantClientAuth(true);
+        } else {
+            sslContext.getDefaultSSLParameters().setWantClientAuth(false);
+        }
+
+        return sslContext;
+
+    }
+
+    /**
+     * Creates a SSLContext instance using the given information. This method assumes that the key password is
+     * the same as the keystore password. If this is not the case, use the {@link #createSslContext(String, char[], char[], String, String)}
+     * method instead.
+     *
+     * @param keystore the full path to the keystore
+     * @param keystorePasswd the keystore password
+     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
+     * @param protocol the protocol to use for the SSL connection
+     *
+     * @return a SSLContext instance
+     * @throws KeyStoreException if any issues accessing the keystore
+     * @throws IOException for any problems loading the keystores
+     * @throws NoSuchAlgorithmException if an algorithm is found to be used but is unknown
+     * @throws CertificateException if there is an issue with the certificate
+     * @throws UnrecoverableKeyException if the key is insufficient
+     * @throws KeyManagementException if unable to manage the key
+     */
+    public static SSLContext createSslContext(
+        final String keystore, final char[] keystorePasswd, final String keystoreType, final String protocol)
+        throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
+        UnrecoverableKeyException, KeyManagementException {
+
+        // create SSL Context passing keystore password as the key password
+        return createSslContext(keystore, keystorePasswd, keystorePasswd, keystoreType, protocol);
+    }
+
+    /**
+     * Creates a SSLContext instance using the given information.
+     *
+     * @param keystore the full path to the keystore
+     * @param keystorePasswd the keystore password
+     * @param keystoreType the type of keystore (e.g., PKCS12, JKS)
+     * @param protocol the protocol to use for the SSL connection
+     *
+     * @return a SSLContext instance
+     * @throws KeyStoreException if any issues accessing the keystore
+     * @throws IOException for any problems loading the keystores
+     * @throws NoSuchAlgorithmException if an algorithm is found to be used but is unknown
+     * @throws CertificateException if there is an issue with the certificate
+     * @throws UnrecoverableKeyException if the key is insufficient
+     * @throws KeyManagementException if unable to manage the key
+     */
+    public static SSLContext createSslContext(
+        final String keystore, final char[] keystorePasswd, final char[] keyPasswd, final String keystoreType, final String protocol)
+            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
+            UnrecoverableKeyException, KeyManagementException {
+
+        // prepare the keystore
+        final KeyStore keyStore = KeyStoreUtils.getKeyStore(keystoreType);
+        try (final InputStream keyStoreStream = new FileInputStream(keystore)) {
+            keyStore.load(keyStoreStream, keystorePasswd);
+        }
+        final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm());
+        if (keyPasswd == null) {
+            keyManagerFactory.init(keyStore, keystorePasswd);
+        } else {
+            keyManagerFactory.init(keyStore, keyPasswd);
+        }
+
+        // initialize the ssl context
+        final SSLContext ctx = SSLContext.getInstance(protocol);
+        ctx.init(keyManagerFactory.getKeyManagers(), new TrustManager[0], new SecureRandom());
+
+        return ctx;
+
+    }
+
+    /**
+     * Creates a SSLContext instance using the given information.
+     *
+     * @param truststore the full path to the truststore
+     * @param truststorePasswd the truststore password
+     * @param truststoreType the type of truststore (e.g., PKCS12, JKS)
+     * @param protocol the protocol to use for the SSL connection
+     *
+     * @return a SSLContext instance
+     * @throws KeyStoreException if any issues accessing the keystore
+     * @throws IOException for any problems loading the keystores
+     * @throws NoSuchAlgorithmException if an algorithm is found to be used but is unknown
+     * @throws CertificateException if there is an issue with the certificate
+     * @throws UnrecoverableKeyException if the key is insufficient
+     * @throws KeyManagementException if unable to manage the key
+     */
+    public static SSLContext createTrustSslContext(
+            final String truststore, final char[] truststorePasswd, final String truststoreType, final String protocol)
+            throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException,
+            UnrecoverableKeyException, KeyManagementException {
+
+        // prepare the truststore
+        final KeyStore trustStore = KeyStoreUtils.getTrustStore(truststoreType);
+        try (final InputStream trustStoreStream = new FileInputStream(truststore)) {
+            trustStore.load(trustStoreStream, truststorePasswd);
+        }
+        final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm());
+        trustManagerFactory.init(trustStore);
+
+        // initialize the ssl context
+        final SSLContext ctx = SSLContext.getInstance(protocol);
+        ctx.init(new KeyManager[0], trustManagerFactory.getTrustManagers(), new SecureRandom());
+
+        return ctx;
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/pom.xml
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/pom.xml b/nifi-registry-web-api/pom.xml
index 0174bd5..ab7dd84 100644
--- a/nifi-registry-web-api/pom.xml
+++ b/nifi-registry-web-api/pom.xml
@@ -31,6 +31,9 @@
             <resource>
                 <directory>src/main/resources</directory>
             </resource>
+            <resource>
+                <directory>src/main/xsd</directory>
+            </resource>
         </resources>
         <plugins>
             <plugin>
@@ -45,6 +48,28 @@
                 </configuration>
             </plugin>
             <plugin>
+                <groupId>org.codehaus.mojo</groupId>
+                <artifactId>jaxb2-maven-plugin</artifactId>
+                <executions>
+                    <execution>
+                        <id>identity-providers</id>
+                        <goals>
+                            <goal>xjc</goal>
+                        </goals>
+                        <configuration>
+                            <packageName>org.apache.nifi.registry.security.authentication.generated</packageName>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-checkstyle-plugin</artifactId>
+                <configuration>
+                    <excludes>**/authentication/generated/*.java,</excludes>
+                </configuration>
+            </plugin>
+            <plugin>
                 <groupId>com.github.kongchen</groupId>
                 <artifactId>swagger-maven-plugin</artifactId>
                 <version>3.1.5</version>
@@ -193,22 +218,27 @@
         </dependency>
         <!-- Test Dependencies -->
         <dependency>
+            <groupId>org.apache.nifi.registry</groupId>
+            <artifactId>nifi-registry-client</artifactId>
+            <version>0.0.1-SNAPSHOT</version>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>org.springframework.boot</groupId>
             <artifactId>spring-boot-starter-test</artifactId>
             <version>${spring.boot.version}</version>
             <scope>test</scope>
         </dependency>
         <dependency>
-            <!-- Integration tests run using embedded Jetty server -->
             <groupId>org.springframework.boot</groupId>
             <artifactId>spring-boot-starter-jetty</artifactId>
             <version>${spring.boot.version}</version>
             <scope>test</scope>
         </dependency>
         <dependency>
-            <groupId>org.apache.nifi.registry</groupId>
-            <artifactId>nifi-registry-client</artifactId>
-            <version>0.0.1-SNAPSHOT</version>
+            <groupId>com.unboundid</groupId>
+            <artifactId>unboundid-ldapsdk</artifactId>
+            <version>3.2.1</version>
             <scope>test</scope>
         </dependency>
     </dependencies>

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryApiApplication.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryApiApplication.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryApiApplication.java
index e8e36a7..19adaed 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryApiApplication.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/NiFiRegistryApiApplication.java
@@ -38,13 +38,13 @@ public class NiFiRegistryApiApplication extends SpringBootServletInitializer {
 
     @Override
     protected SpringApplicationBuilder configure(SpringApplicationBuilder application) {
-        final Properties fixedProps = new Properties();
-        fixedProps.setProperty("spring.jpa.hibernate.ddl-auto", "none");
-        fixedProps.setProperty("spring.jpa.hibernate.naming.physical-strategy", "org.hibernate.boot.model.naming.PhysicalNamingStrategyStandardImpl");
+        final Properties defaultProperties = new Properties();
+        defaultProperties.setProperty("spring.jpa.hibernate.ddl-auto", "none");
+        defaultProperties.setProperty("spring.jpa.hibernate.naming.physical-strategy", "org.hibernate.boot.model.naming.PhysicalNamingStrategyStandardImpl");
 
         return application
                 .sources(NiFiRegistryApiApplication.class)
-                .properties(fixedProps);
+                .properties(defaultProperties);
     }
 
     public static void main(String[] args) {

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/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 d577e1e..503c948 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
@@ -29,8 +29,13 @@ import org.glassfish.jersey.server.ServerProperties;
 import org.glassfish.jersey.server.filter.HttpMethodOverrideFilter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.web.servlet.FilterRegistrationBean;
+import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.Configuration;
+import org.springframework.web.filter.HiddenHttpMethodFilter;
 
+import javax.servlet.Filter;
 import javax.servlet.ServletContext;
 import javax.ws.rs.core.Context;
 
@@ -63,4 +68,12 @@ public class NiFiRegistryResourceConfig extends ResourceConfig {
 
         property(ServerProperties.BV_SEND_ERROR_IN_RESPONSE, true);
     }
+
+    // Disable default SpringMVC filter beans that are not compatible with Jersey
+    @Bean
+    public FilterRegistrationBean registration(@Autowired HiddenHttpMethodFilter filter) {
+        FilterRegistrationBean registration = new FilterRegistrationBean((Filter) filter);
+        registration.setEnabled(false);
+        return registration;
+    }
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistrySecurityConfig.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistrySecurityConfig.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistrySecurityConfig.java
deleted file mode 100644
index 7652e90..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/NiFiRegistrySecurityConfig.java
+++ /dev/null
@@ -1,221 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web;
-
-import org.apache.nifi.registry.authorization.AuthorizableLookup;
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.AuthorizerFactory;
-import org.apache.nifi.registry.authorization.StandardAuthorizableLookup;
-import org.apache.nifi.registry.authorization.StandardAuthorizerFactory;
-import org.apache.nifi.registry.extension.ExtensionManager;
-import org.apache.nifi.registry.properties.NiFiRegistryProperties;
-import org.apache.nifi.registry.web.security.NiFiAnonymousUserFilter;
-import org.apache.nifi.registry.web.security.x509.SubjectDnX509PrincipalExtractor;
-import org.apache.nifi.registry.web.security.x509.X509AuthenticationFilter;
-import org.apache.nifi.registry.web.security.x509.X509AuthenticationProvider;
-import org.apache.nifi.registry.web.security.x509.X509CertificateExtractor;
-import org.apache.nifi.registry.web.security.x509.X509CertificateValidator;
-import org.apache.nifi.registry.web.security.x509.X509IdentityProvider;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.context.annotation.Bean;
-import org.springframework.context.annotation.Configuration;
-import org.springframework.security.authentication.AuthenticationManager;
-import org.springframework.security.config.annotation.authentication.builders.AuthenticationManagerBuilder;
-import org.springframework.security.config.annotation.method.configuration.EnableGlobalMethodSecurity;
-import org.springframework.security.config.annotation.web.builders.HttpSecurity;
-import org.springframework.security.config.annotation.web.builders.WebSecurity;
-import org.springframework.security.config.annotation.web.configuration.EnableWebSecurity;
-import org.springframework.security.config.annotation.web.configuration.WebSecurityConfigurerAdapter;
-import org.springframework.security.config.http.SessionCreationPolicy;
-import org.springframework.security.web.authentication.AnonymousAuthenticationFilter;
-import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
-
-/**
- * NiFi Web Api Spring security
- */
-@Configuration
-@EnableWebSecurity
-@EnableGlobalMethodSecurity(prePostEnabled = true)
-public class NiFiRegistrySecurityConfig extends WebSecurityConfigurerAdapter {
-    private static final Logger logger = LoggerFactory.getLogger(NiFiRegistrySecurityConfig.class);
-
-    private final NiFiRegistryProperties properties;
-    private final ExtensionManager extensionManager;
-
-    private X509AuthenticationFilter x509AuthenticationFilter;
-
-//    private JwtAuthenticationFilter jwtAuthenticationFilter;
-//    private JwtAuthenticationProvider jwtAuthenticationProvider;
-//
-//    private OtpAuthenticationFilter otpAuthenticationFilter;
-//    private OtpAuthenticationProvider otpAuthenticationProvider;
-
-    private NiFiAnonymousUserFilter anonymousAuthenticationFilter;
-
-    @Autowired
-    public NiFiRegistrySecurityConfig(final NiFiRegistryProperties properties, final ExtensionManager extensionManager) {
-        super(true); // disable defaults
-        this.properties = properties;
-        this.extensionManager = extensionManager;
-    }
-
-    @Override
-    public void configure(WebSecurity webSecurity) throws Exception {
-        // ignore the access endpoints for obtaining the access config, access token
-        // granting, and access status for a given user (note: we are not ignoring the
-        // the /access/download-token endpoints)
-        webSecurity
-                .ignoring()
-                    .antMatchers("/access", "/access/config", "/access/token");
-    }
-
-    @Override
-    protected void configure(HttpSecurity http) throws Exception {
-        http
-                .rememberMe().disable()
-                .authorizeRequests()
-                    .anyRequest().fullyAuthenticated()
-                    .and()
-                .sessionManagement()
-                    .sessionCreationPolicy(SessionCreationPolicy.STATELESS);
-
-        // x509
-        http.addFilterBefore(x509AuthenticationFilter(), AnonymousAuthenticationFilter.class);
-
-        // jwt
-        // http.addFilterBefore(jwtFilterBean(), AnonymousAuthenticationFilter.class);
-
-        // otp
-        // http.addFilterBefore(otpFilterBean(), AnonymousAuthenticationFilter.class);
-
-        // anonymous
-        http.anonymous().authenticationFilter(anonymousFilter());
-    }
-
-    @Bean
-    @Override
-    public AuthenticationManager authenticationManagerBean() throws Exception {
-        // override xxxBean method so the authentication manager is available in app context (necessary for the method level security)
-        return super.authenticationManagerBean();
-    }
-
-    @Override
-    protected void configure(AuthenticationManagerBuilder auth) throws Exception {
-        auth
-                .authenticationProvider(x509AuthenticationProvider());
-        /* TODO, add Jwt and Otp support */
-//                .authenticationProvider(jwtAuthenticationProvider)
-//                .authenticationProvider(otpAuthenticationProvider);
-    }
-
-//    @Bean // TODO JwtAuthenticationFilter
-//    public JwtAuthenticationFilter jwtFilterBean() throws Exception {
-//        if (jwtAuthenticationFilter == null) {
-//            jwtAuthenticationFilter = new JwtAuthenticationFilter();
-//            jwtAuthenticationFilter.setProperties(properties);
-//            jwtAuthenticationFilter.setAuthenticationManager(authenticationManager());
-//        }
-//        return jwtAuthenticationFilter;
-//    }
-//
-//    @Bean // TODO OtpAuthenticationFilter
-//    public OtpAuthenticationFilter otpFilterBean() throws Exception {
-//        if (otpAuthenticationFilter == null) {
-//            otpAuthenticationFilter = new OtpAuthenticationFilter();
-//            otpAuthenticationFilter.setProperties(properties);
-//            otpAuthenticationFilter.setAuthenticationManager(authenticationManager());
-//        }
-//        return otpAuthenticationFilter;
-//    }
-
-    @Bean
-    public X509AuthenticationFilter x509AuthenticationFilter() throws Exception {
-        if (x509AuthenticationFilter == null) {
-            x509AuthenticationFilter = new X509AuthenticationFilter();
-            x509AuthenticationFilter.setProperties(properties);
-            x509AuthenticationFilter.setCertificateExtractor(certificateExtractor());
-            x509AuthenticationFilter.setPrincipalExtractor(principalExtractor());
-            x509AuthenticationFilter.setAuthenticationManager(authenticationManager());
-        }
-        return x509AuthenticationFilter;
-    }
-
-    @Bean
-    public NiFiAnonymousUserFilter anonymousFilter() throws Exception {
-        if (anonymousAuthenticationFilter == null) {
-            anonymousAuthenticationFilter = new NiFiAnonymousUserFilter();
-        }
-        return anonymousAuthenticationFilter;
-    }
-
-    @Bean
-    public X509CertificateExtractor certificateExtractor() {
-        return new X509CertificateExtractor();
-    }
-
-    public X509CertificateValidator certificateValidator() {
-        return new X509CertificateValidator();
-    }
-
-    @Bean
-    public X509PrincipalExtractor principalExtractor() {
-        return new SubjectDnX509PrincipalExtractor();
-    }
-
-    @Bean
-    public X509IdentityProvider x509IdentityProvider() {
-        X509IdentityProvider x509IdentityProvider = new X509IdentityProvider();
-        x509IdentityProvider.setCertificateValidator(certificateValidator());
-        x509IdentityProvider.setPrincipalExtractor(principalExtractor());
-        return x509IdentityProvider;
-    }
-
-    @Bean
-    public X509AuthenticationProvider x509AuthenticationProvider() {
-        return new X509AuthenticationProvider(x509IdentityProvider(), authorizer(), this.properties);
-    }
-
-    @Bean(initMethod = "initialize")
-    public AuthorizerFactory authorizerFactory() {
-        return new StandardAuthorizerFactory(this.properties, this.extensionManager);
-    }
-
-    @Bean
-    public Authorizer authorizer() {
-        return authorizerFactory().getAuthorizer();
-    }
-
-    @Bean
-    public AuthorizableLookup authorizableLookup() {
-        return new StandardAuthorizableLookup();
-    }
-
-
-    // TODO, add Jwt and Otp support
-//    @Autowired
-//    public void setJwtAuthenticationProvider(JwtAuthenticationProvider jwtAuthenticationProvider) {
-//        this.jwtAuthenticationProvider = jwtAuthenticationProvider;
-//    }
-//
-//    @Autowired
-//    public void setOtpAuthenticationProvider(OtpAuthenticationProvider otpAuthenticationProvider) {
-//        this.otpAuthenticationProvider = otpAuthenticationProvider;
-//    }
-
-}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessPolicyResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessPolicyResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessPolicyResource.java
index c78e84d..b9cfbb9 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessPolicyResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AccessPolicyResource.java
@@ -35,11 +35,11 @@ import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiParam;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.AuthorizerCapabilityDetection;
-import org.apache.nifi.registry.authorization.RequestAction;
-import org.apache.nifi.registry.authorization.resource.Authorizable;
-import org.apache.nifi.registry.authorization.user.NiFiUserUtils;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.AuthorizerCapabilityDetection;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.resource.Authorizable;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
 import org.apache.nifi.registry.model.authorization.AccessPolicy;
 import org.apache.nifi.registry.model.authorization.AccessPolicySummary;
 import org.apache.nifi.registry.service.AuthorizationService;
@@ -91,7 +91,7 @@ public class AccessPolicyResource extends AuthorizableApplicationResource {
             @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) })
+            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409 + " The NiFi Registry might not be configured to use a ConfigurableAccessPolicyProvider.") })
     public Response createAccessPolicy(
             @Context final HttpServletRequest httpServletRequest,
             @ApiParam(value = "The access policy configuration details.", required = true)
@@ -163,7 +163,8 @@ public class AccessPolicyResource extends AuthorizableApplicationResource {
     @ApiResponses({
             @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401),
             @ApiResponse(code = 403, message = HttpStatusMessages.MESSAGE_403),
-            @ApiResponse(code = 404, message = HttpStatusMessages.MESSAGE_404) })
+            @ApiResponse(code = 404, message = HttpStatusMessages.MESSAGE_404),
+            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409) })
     public Response getAccessPolicy(
             @ApiParam(value = "The access policy id.", required = true)
             @PathParam("id") final String identifier) {
@@ -245,7 +246,7 @@ public class AccessPolicyResource extends AuthorizableApplicationResource {
             @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401),
             @ApiResponse(code = 403, message = HttpStatusMessages.MESSAGE_403),
             @ApiResponse(code = 404, message = HttpStatusMessages.MESSAGE_404),
-            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409) })
+            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409 + " The NiFi Registry might not be configured to use a ConfigurableAccessPolicyProvider.") })
     public Response updateAccessPolicy(
             @Context
             final HttpServletRequest httpServletRequest,
@@ -290,9 +291,10 @@ public class AccessPolicyResource extends AuthorizableApplicationResource {
             response = AccessPolicy.class
     )
     @ApiResponses({
-            @ApiResponse(code = 401, message = "Client could not be authenticated."),
-            @ApiResponse(code = 403, message = "Client is not authorized to make this request."),
-            @ApiResponse(code = 404, message = "The specified resource could not be found.") })
+            @ApiResponse(code = 401, message = HttpStatusMessages.MESSAGE_401),
+            @ApiResponse(code = 403, message = HttpStatusMessages.MESSAGE_403),
+            @ApiResponse(code = 404, message = HttpStatusMessages.MESSAGE_404),
+            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409 + " The NiFi Registry might not be configured to use a ConfigurableAccessPolicyProvider.") })
     public Response removeAccessPolicy(
             @Context final HttpServletRequest httpServletRequest,
             @ApiParam(value = "The access policy id.", required = true)
@@ -314,6 +316,7 @@ public class AccessPolicyResource extends AuthorizableApplicationResource {
 
     private void verifyAuthorizerSupportsConfigurablePolicies() {
         if (!AuthorizerCapabilityDetection.isConfigurableAccessPolicyProvider(authorizer)) {
+            verifyAuthorizerIsManaged();
             throw new IllegalStateException(AuthorizationService.MSG_NON_CONFIGURABLE_POLICIES);
         }
     }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/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 a5c849a..ad923c4 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,38 +16,54 @@
  */
 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;
 import io.swagger.annotations.ApiResponses;
-import org.apache.commons.lang3.NotImplementedException;
-import org.apache.nifi.registry.authorization.exception.AccessDeniedException;
-import org.apache.nifi.registry.authorization.user.NiFiUser;
-import org.apache.nifi.registry.authorization.user.NiFiUserDetails;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.security.authentication.AuthenticationResponse;
+import org.apache.nifi.registry.security.authentication.LoginCredentials;
+import org.apache.nifi.registry.security.authentication.LoginIdentityProvider;
+import org.apache.nifi.registry.security.authentication.exception.IdentityAccessException;
+import org.apache.nifi.registry.security.authentication.exception.InvalidLoginCredentialsException;
+import org.apache.nifi.registry.security.authorization.exception.AccessDeniedException;
+import org.apache.nifi.registry.security.authorization.user.NiFiUser;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserDetails;
 import org.apache.nifi.registry.exception.AdministrationException;
 import org.apache.nifi.registry.model.authorization.AccessStatus;
-import org.apache.nifi.registry.web.security.InvalidAuthenticationException;
-import org.apache.nifi.registry.web.security.ProxiedEntitiesUtils;
-import org.apache.nifi.registry.web.security.UntrustedProxyException;
-import org.apache.nifi.registry.web.security.token.NiFiAuthenticationToken;
-import org.apache.nifi.registry.web.security.x509.X509AuthenticationProvider;
-import org.apache.nifi.registry.web.security.x509.X509AuthenticationRequestToken;
-import org.apache.nifi.registry.web.security.x509.X509CertificateExtractor;
+import org.apache.nifi.registry.web.security.authentication.exception.InvalidAuthenticationException;
+import org.apache.nifi.registry.web.security.authentication.ProxiedEntitiesUtils;
+import org.apache.nifi.registry.web.security.authentication.exception.UntrustedProxyException;
+import org.apache.nifi.registry.web.security.authentication.jwt.JwtAuthenticationFilter;
+import org.apache.nifi.registry.web.security.authentication.jwt.JwtAuthenticationProvider;
+import org.apache.nifi.registry.web.security.authentication.jwt.JwtAuthenticationRequestToken;
+import org.apache.nifi.registry.web.security.authentication.jwt.JwtService;
+import org.apache.nifi.registry.web.security.authentication.token.LoginAuthenticationToken;
+import org.apache.nifi.registry.web.security.authentication.token.NiFiAuthenticationToken;
+import org.apache.nifi.registry.web.security.authentication.x509.X509AuthenticationProvider;
+import org.apache.nifi.registry.web.security.authentication.x509.X509AuthenticationRequestToken;
+import org.apache.nifi.registry.web.security.authentication.x509.X509CertificateExtractor;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.security.authentication.AuthenticationServiceException;
 import org.springframework.security.web.authentication.preauth.x509.X509PrincipalExtractor;
 import org.springframework.stereotype.Component;
 
 import javax.servlet.http.HttpServletRequest;
 import javax.ws.rs.Consumes;
+import javax.ws.rs.FormParam;
 import javax.ws.rs.GET;
+import javax.ws.rs.POST;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
 import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
+import java.net.URI;
 import java.security.cert.X509Certificate;
+import java.util.concurrent.TimeUnit;
 
 @Component
 @Path("/access")
@@ -62,11 +78,24 @@ public class AccessResource extends ApplicationResource {
     private X509CertificateExtractor certificateExtractor;
     private X509AuthenticationProvider x509AuthenticationProvider;
     private X509PrincipalExtractor x509principalExtractor;
+    private LoginIdentityProvider loginIdentityProvider;
+    private JwtAuthenticationProvider jwtAuthenticationProvider;
+    private JwtService jwtService;
 
-    public AccessResource(X509CertificateExtractor certificateExtractor, X509AuthenticationProvider x509AuthenticationProvider, X509PrincipalExtractor x509principalExtractor) {
+    @Autowired
+    public AccessResource(X509CertificateExtractor certificateExtractor,
+                          X509AuthenticationProvider x509AuthenticationProvider,
+                          X509PrincipalExtractor x509principalExtractor,
+                          LoginIdentityProvider loginIdentityProvider,
+                          JwtAuthenticationProvider jwtAuthenticationProvider,
+                          JwtService jwtService
+    ) {
         this.certificateExtractor = certificateExtractor;
         this.x509AuthenticationProvider = x509AuthenticationProvider;
         this.x509principalExtractor = x509principalExtractor;
+        this.loginIdentityProvider = loginIdentityProvider;
+        this.jwtAuthenticationProvider = jwtAuthenticationProvider;
+        this.jwtService = jwtService;
     }
 
     /**
@@ -78,16 +107,15 @@ public class AccessResource extends ApplicationResource {
     @GET
     @Consumes(MediaType.WILDCARD)
     @Produces(MediaType.APPLICATION_JSON)
-    @Path("")
     @ApiOperation(
             value = "Gets the status the client's access",
             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) })
+            @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
         if (!httpServletRequest.isSecure()) {
@@ -102,36 +130,32 @@ public class AccessResource extends ApplicationResource {
             // if there is not certificate, consider a token
             if (certificates == null) {
 
-                // TODO - add JWT Authentication support
-                throw new NotImplementedException("NiFi Registry client is trying to authentication with something other than a client cert. " +
-                        "At this time, only client certificate authentication is supported.");
-
-//                // look for an authorization token
-//                final String authorization = httpServletRequest.getHeader(JwtAuthenticationFilter.AUTHORIZATION);
-//
-//                // if there is no authorization header, we don't know the user
-//                if (authorization == null) {
-//                    accessStatus.setStatus(AccessStatus.Status.UNKNOWN.name());
-//                    accessStatus.setMessage("No credentials supplied, unknown user.");
-//                } else {
-//                    try {
-//                        // Extract the Base64 encoded token from the Authorization header
-//                        final String token = StringUtils.substringAfterLast(authorization, " ");
-//
-//                        final JwtAuthenticationRequestToken jwtRequest = new JwtAuthenticationRequestToken(token, httpServletRequest.getRemoteAddr());
-//                        final NiFiAuthenticationToken authenticationResponse = (NiFiAuthenticationToken) jwtAuthenticationProvider.authenticate(jwtRequest);
-//                        final NiFiUser nifiUser = ((NiFiUserDetails) authenticationResponse.getDetails()).getNiFiUser();
-//
-//                        // set the user identity
-//                        accessStatus.setIdentity(nifiUser.getIdentity());
-//
-//                        // attempt authorize to /flow
-//                        accessStatus.setStatus(AccessStatus.Status.ACTIVE.name());
-//                        accessStatus.setMessage("You are already logged in.");
-//                    } catch (JwtException e) {
-//                        throw new InvalidAuthenticationException(e.getMessage(), e);
-//                    }
-//                }
+                // look for an authorization token
+                final String authorization = httpServletRequest.getHeader(JwtAuthenticationFilter.AUTHORIZATION);
+
+                // if there is no authorization header, we don't know the user
+                if (authorization == null) {
+                    accessStatus.setStatus(AccessStatus.Status.UNKNOWN.name());
+                    accessStatus.setMessage("No credentials supplied, unknown user.");
+                } else {
+                    try {
+                        // Extract the Base64 encoded token from the Authorization header
+                        final String token = StringUtils.substringAfterLast(authorization, " ");
+
+                        final JwtAuthenticationRequestToken jwtRequest = new JwtAuthenticationRequestToken(token, httpServletRequest.getRemoteAddr());
+                        final NiFiAuthenticationToken authenticationResponse = (NiFiAuthenticationToken) jwtAuthenticationProvider.authenticate(jwtRequest);
+                        final NiFiUser nifiUser = ((NiFiUserDetails) authenticationResponse.getDetails()).getNiFiUser();
+
+                        // set the user identity
+                        accessStatus.setIdentity(nifiUser.getIdentity());
+
+                        // attempt authorize to /flow
+                        accessStatus.setStatus(AccessStatus.Status.ACTIVE.name());
+                        accessStatus.setMessage("You are already logged in.");
+                    } catch (JwtException e) {
+                        throw new InvalidAuthenticationException(e.getMessage(), e);
+                    }
+                }
             } else {
                 try {
                     final X509AuthenticationRequestToken x509Request = new X509AuthenticationRequestToken(
@@ -159,6 +183,91 @@ public class AccessResource extends ApplicationResource {
         return generateOkResponse(accessStatus).build();
     }
 
+    /**
+     * Creates a token for accessing the REST API via username/password.
+     *
+     * @param httpServletRequest the servlet request
+     * @param username           the username
+     * @param password           the password
+     * @return A JWT (string)
+     */
+    @POST
+    @Consumes(MediaType.APPLICATION_FORM_URLENCODED)
+    @Produces(MediaType.TEXT_PLAIN)
+    @Path("/token")
+    @ApiOperation(
+            value = "Creates a token for accessing the REST API via username/password",
+            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 = 403, message = HttpStatusMessages.MESSAGE_403),
+            @ApiResponse(code = 409, message = HttpStatusMessages.MESSAGE_409 + " The NiFi Registry may not be configured to support username/password login."),
+            @ApiResponse(code = 500, message = HttpStatusMessages.MESSAGE_500) })
+    public Response createAccessToken(
+            @Context HttpServletRequest httpServletRequest,
+            @FormParam("username") String username,
+            @FormParam("password") String password) {
+
+        // only support access tokens when communicating over HTTPS
+        if (!httpServletRequest.isSecure()) {
+            throw new IllegalStateException("Access tokens are only issued over HTTPS");
+        }
+
+        // if not configuration for login, don't consider credentials
+        if (loginIdentityProvider == null) {
+            throw new IllegalStateException("Username/Password login not supported by this NiFi");
+        }
+
+        final LoginAuthenticationToken loginAuthenticationToken;
+
+        // ensure we have login credentials
+        if (StringUtils.isBlank(username) || StringUtils.isBlank(password)) {
+            throw new IllegalArgumentException("The username and password must be specified");
+        }
+
+        try {
+            // attempt to authenticate
+            final AuthenticationResponse authenticationResponse = loginIdentityProvider.authenticate(new LoginCredentials(username, password));
+            long expiration = validateTokenExpiration(authenticationResponse.getExpiration(), authenticationResponse.getIdentity());
+
+            // create the authentication token
+            loginAuthenticationToken = new LoginAuthenticationToken(authenticationResponse.getIdentity(), expiration, authenticationResponse.getIssuer());
+        } catch (final InvalidLoginCredentialsException ilce) {
+            throw new IllegalArgumentException("The supplied username and password are not valid.", ilce);
+        } catch (final IdentityAccessException iae) {
+            throw new AdministrationException(iae.getMessage(), iae);
+        }
+
+        // generate JWT for response
+        final String token = jwtService.generateSignedToken(loginAuthenticationToken);
+
+        // build the response
+        final URI uri = URI.create(generateResourceUri("access", "token"));
+        return generateCreatedResponse(uri, token).build();
+    }
+
+    private long validateTokenExpiration(long proposedTokenExpiration, String identity) {
+        final long maxExpiration = TimeUnit.MILLISECONDS.convert(12, TimeUnit.HOURS);
+        final long minExpiration = TimeUnit.MILLISECONDS.convert(1, TimeUnit.MINUTES);
+
+        if (proposedTokenExpiration > maxExpiration) {
+            logger.warn(String.format("Max token expiration exceeded. Setting expiration to %s from %s for %s", maxExpiration,
+                    proposedTokenExpiration, identity));
+            proposedTokenExpiration = maxExpiration;
+        } else if (proposedTokenExpiration < minExpiration) {
+            logger.warn(String.format("Min token expiration not met. Setting expiration to %s from %s for %s", minExpiration,
+                    proposedTokenExpiration, identity));
+            proposedTokenExpiration = minExpiration;
+        }
+
+        return proposedTokenExpiration;
+    }
+
 
 
 }

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AuthorizableApplicationResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AuthorizableApplicationResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AuthorizableApplicationResource.java
index d15685d..58ee742 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AuthorizableApplicationResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/AuthorizableApplicationResource.java
@@ -16,11 +16,11 @@
  */
 package org.apache.nifi.registry.web.api;
 
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.RequestAction;
-import org.apache.nifi.registry.authorization.resource.Authorizable;
-import org.apache.nifi.registry.authorization.resource.ResourceType;
-import org.apache.nifi.registry.authorization.user.NiFiUserUtils;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.resource.Authorizable;
+import org.apache.nifi.registry.security.authorization.resource.ResourceType;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
 import org.apache.nifi.registry.bucket.BucketItem;
 import org.apache.nifi.registry.model.authorization.Resource;
 import org.apache.nifi.registry.service.AuthorizationService;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketFlowResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketFlowResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketFlowResource.java
index f626b3d..f2a419d 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketFlowResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketFlowResource.java
@@ -22,8 +22,8 @@ import io.swagger.annotations.ApiParam;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.RequestAction;
 import org.apache.nifi.registry.bucket.BucketItem;
 import org.apache.nifi.registry.exception.ResourceNotFoundException;
 import org.apache.nifi.registry.flow.VersionedFlow;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketResource.java
index 8179b70..5835ea0 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/BucketResource.java
@@ -22,10 +22,10 @@ import io.swagger.annotations.ApiParam;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.RequestAction;
-import org.apache.nifi.registry.authorization.resource.Authorizable;
-import org.apache.nifi.registry.authorization.user.NiFiUserUtils;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.resource.Authorizable;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
 import org.apache.nifi.registry.bucket.Bucket;
 import org.apache.nifi.registry.bucket.BucketItem;
 import org.apache.nifi.registry.service.AuthorizationService;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/FlowResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/FlowResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/FlowResource.java
index 6e7f347..ce11b0d 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/FlowResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/FlowResource.java
@@ -18,7 +18,7 @@ package org.apache.nifi.registry.web.api;
 
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
-import org.apache.nifi.registry.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.Authorizer;
 import org.apache.nifi.registry.field.Fields;
 import org.apache.nifi.registry.service.AuthorizationService;
 import org.apache.nifi.registry.service.RegistryService;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ItemResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ItemResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ItemResource.java
index 458f0f0..8b8a3ed 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ItemResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ItemResource.java
@@ -19,8 +19,8 @@ package org.apache.nifi.registry.web.api;
 import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiParam;
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.RequestAction;
 import org.apache.nifi.registry.bucket.BucketItem;
 import org.apache.nifi.registry.service.AuthorizationService;
 import org.apache.nifi.registry.service.RegistryService;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ResourceResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ResourceResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ResourceResource.java
index bcef565..d0cb879 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ResourceResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/ResourceResource.java
@@ -20,10 +20,10 @@ import io.swagger.annotations.Api;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.RequestAction;
-import org.apache.nifi.registry.authorization.resource.Authorizable;
-import org.apache.nifi.registry.authorization.user.NiFiUserUtils;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.resource.Authorizable;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
 import org.apache.nifi.registry.service.AuthorizationService;
 import org.apache.nifi.registry.model.authorization.Resource;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/TenantResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/TenantResource.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/TenantResource.java
index f6fd15f..3d088d7 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/TenantResource.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/api/TenantResource.java
@@ -22,11 +22,11 @@ import io.swagger.annotations.ApiParam;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.authorization.Authorizer;
-import org.apache.nifi.registry.authorization.AuthorizerCapabilityDetection;
-import org.apache.nifi.registry.authorization.RequestAction;
-import org.apache.nifi.registry.authorization.resource.Authorizable;
-import org.apache.nifi.registry.authorization.user.NiFiUserUtils;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.AuthorizerCapabilityDetection;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.resource.Authorizable;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
 import org.apache.nifi.registry.model.authorization.User;
 import org.apache.nifi.registry.model.authorization.UserGroup;
 import org.apache.nifi.registry.service.AuthorizationService;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/AccessDeniedExceptionMapper.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/AccessDeniedExceptionMapper.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/AccessDeniedExceptionMapper.java
index 13ed44f..5b9e3ee 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/AccessDeniedExceptionMapper.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/AccessDeniedExceptionMapper.java
@@ -17,9 +17,9 @@
 package org.apache.nifi.registry.web.mapper;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.authorization.exception.AccessDeniedException;
-import org.apache.nifi.registry.authorization.user.NiFiUser;
-import org.apache.nifi.registry.authorization.user.NiFiUserUtils;
+import org.apache.nifi.registry.security.authorization.exception.AccessDeniedException;
+import org.apache.nifi.registry.security.authorization.user.NiFiUser;
+import org.apache.nifi.registry.security.authorization.user.NiFiUserUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.stereotype.Component;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/AuthorizationAccessExceptionMapper.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/AuthorizationAccessExceptionMapper.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/AuthorizationAccessExceptionMapper.java
index cc829ce..ff4b7ec 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/AuthorizationAccessExceptionMapper.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/AuthorizationAccessExceptionMapper.java
@@ -16,7 +16,7 @@
  */
 package org.apache.nifi.registry.web.mapper;
 
-import org.apache.nifi.registry.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.stereotype.Component;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/InvalidAuthenticationExceptionMapper.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/InvalidAuthenticationExceptionMapper.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/InvalidAuthenticationExceptionMapper.java
index 74af1df..15bb227 100644
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/InvalidAuthenticationExceptionMapper.java
+++ b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/mapper/InvalidAuthenticationExceptionMapper.java
@@ -17,7 +17,7 @@
 package org.apache.nifi.registry.web.mapper;
 
 import org.apache.commons.lang3.StringUtils;
-import org.apache.nifi.registry.web.security.InvalidAuthenticationException;
+import org.apache.nifi.registry.web.security.authentication.exception.InvalidAuthenticationException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.springframework.stereotype.Component;

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/InvalidAuthenticationException.java
----------------------------------------------------------------------
diff --git a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/InvalidAuthenticationException.java b/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/InvalidAuthenticationException.java
deleted file mode 100644
index c7a1aea..0000000
--- a/nifi-registry-web-api/src/main/java/org/apache/nifi/registry/web/security/InvalidAuthenticationException.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.nifi.registry.web.security;
-
-import org.springframework.security.core.AuthenticationException;
-
-/**
- * Thrown if the authentication of a given request is invalid. For instance,
- * an expired certificate or token.
- */
-public class InvalidAuthenticationException extends AuthenticationException {
-
-    public InvalidAuthenticationException(String msg) {
-        super(msg);
-    }
-
-    public InvalidAuthenticationException(String msg, Throwable t) {
-        super(msg, t);
-    }
-
-}

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

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

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

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


[12/17] nifi-registry git commit: NIFIREG-33 Add LDAP and JWT auth support

Posted by bb...@apache.org.
http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileUserGroupProvider.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileUserGroupProvider.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileUserGroupProvider.java
new file mode 100644
index 0000000..85d62cd
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/FileUserGroupProvider.java
@@ -0,0 +1,775 @@
+/*
+ * 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.authorization.file;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.security.authorization.annotation.AuthorizerContext;
+import org.apache.nifi.registry.security.authorization.file.tenants.generated.Groups;
+import org.apache.nifi.registry.security.authorization.file.tenants.generated.Tenants;
+import org.apache.nifi.registry.security.authorization.file.tenants.generated.Users;
+import org.apache.nifi.registry.properties.util.IdentityMapping;
+import org.apache.nifi.registry.properties.util.IdentityMappingUtil;
+import org.apache.nifi.registry.util.PropertyValue;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizationAccessException;
+import org.apache.nifi.registry.security.authorization.AuthorizerConfigurationContext;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerCreationException;
+import org.apache.nifi.registry.security.authorization.exception.AuthorizerDestructionException;
+import org.apache.nifi.registry.security.authorization.ConfigurableUserGroupProvider;
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.exception.UninheritableAuthorizationsException;
+import org.apache.nifi.registry.security.authorization.User;
+import org.apache.nifi.registry.security.authorization.UserAndGroups;
+import org.apache.nifi.registry.security.authorization.UserGroupProviderInitializationContext;
+import org.apache.nifi.registry.properties.NiFiRegistryProperties;
+import org.apache.nifi.registry.util.FileUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.xml.sax.SAXException;
+
+import javax.xml.XMLConstants;
+import javax.xml.bind.JAXBContext;
+import javax.xml.bind.JAXBElement;
+import javax.xml.bind.JAXBException;
+import javax.xml.bind.Marshaller;
+import javax.xml.bind.Unmarshaller;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.stream.XMLOutputFactory;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamWriter;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.validation.Schema;
+import javax.xml.validation.SchemaFactory;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class FileUserGroupProvider implements ConfigurableUserGroupProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(FileUserGroupProvider.class);
+
+    private static final String TENANTS_XSD = "/tenants.xsd";
+    private static final String JAXB_TENANTS_PATH = "org.apache.nifi.registry.security.authorization.file.tenants.generated";
+
+    private static final JAXBContext JAXB_TENANTS_CONTEXT = initializeJaxbContext(JAXB_TENANTS_PATH);
+
+    /**
+     * Load the JAXBContext.
+     */
+    private static JAXBContext initializeJaxbContext(final String contextPath) {
+        try {
+            return JAXBContext.newInstance(contextPath, FileAuthorizer.class.getClassLoader());
+            //return JAXBContext.newInstance(contextPath);
+        } catch (JAXBException e) {
+            throw new RuntimeException("Unable to create JAXBContext: " + e);
+        }
+    }
+
+    private static final DocumentBuilderFactory DOCUMENT_BUILDER_FACTORY = DocumentBuilderFactory.newInstance();
+    private static final XMLOutputFactory XML_OUTPUT_FACTORY = XMLOutputFactory.newInstance();
+
+    private static final String USER_ELEMENT = "user";
+    private static final String GROUP_USER_ELEMENT = "groupUser";
+    private static final String GROUP_ELEMENT = "group";
+    private static final String IDENTIFIER_ATTR = "identifier";
+    private static final String IDENTITY_ATTR = "identity";
+    private static final String NAME_ATTR = "name";
+
+    static final String PROP_INITIAL_USER_IDENTITY_PREFIX = "Initial User Identity ";
+    static final String PROP_TENANTS_FILE = "Users File";
+    static final Pattern INITIAL_USER_IDENTITY_PATTERN = Pattern.compile(PROP_INITIAL_USER_IDENTITY_PREFIX + "\\S+");
+
+    private Schema usersSchema;
+    private Schema tenantsSchema;
+    private NiFiRegistryProperties properties;
+    private File tenantsFile;
+    private File restoreTenantsFile;
+    private Set<String> initialUserIdentities;
+    private List<IdentityMapping> identityMappings;
+
+    private final AtomicReference<UserGroupHolder> userGroupHolder = new AtomicReference<>();
+
+    @Override
+    public void initialize(UserGroupProviderInitializationContext initializationContext) throws AuthorizerCreationException {
+        try {
+            final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI);
+            tenantsSchema = schemaFactory.newSchema(FileAuthorizer.class.getResource(TENANTS_XSD));
+            //usersSchema = schemaFactory.newSchema(FileAuthorizer.class.getResource(USERS_XSD));
+        } catch (Exception e) {
+            throw new AuthorizerCreationException(e);
+        }
+    }
+
+    @Override
+    public void onConfigured(AuthorizerConfigurationContext configurationContext) throws AuthorizerCreationException {
+        try {
+            final PropertyValue tenantsPath = configurationContext.getProperty(PROP_TENANTS_FILE);
+            if (StringUtils.isBlank(tenantsPath.getValue())) {
+                throw new AuthorizerCreationException("The users file must be specified.");
+            }
+
+            // get the tenants file and ensure it exists
+            tenantsFile = new File(tenantsPath.getValue());
+            if (!tenantsFile.exists()) {
+                logger.info("Creating new users file at {}", new Object[] {tenantsFile.getAbsolutePath()});
+                saveTenants(new Tenants());
+            }
+
+            final File tenantsFileDirectory = tenantsFile.getAbsoluteFile().getParentFile();
+
+            // extract the identity mappings from nifi.properties if any are provided
+            identityMappings = Collections.unmodifiableList(IdentityMappingUtil.getIdentityMappings(properties));
+
+            // extract any node identities
+            initialUserIdentities = new HashSet<>();
+            for (Map.Entry<String,String> entry : configurationContext.getProperties().entrySet()) {
+                Matcher matcher = INITIAL_USER_IDENTITY_PATTERN.matcher(entry.getKey());
+                if (matcher.matches() && !StringUtils.isBlank(entry.getValue())) {
+                    initialUserIdentities.add(IdentityMappingUtil.mapIdentity(entry.getValue(), identityMappings));
+                }
+            }
+
+            load();
+
+            // if we've copied the authorizations file to a restore directory synchronize it
+            if (restoreTenantsFile != null) {
+                FileUtils.copyFile(tenantsFile, restoreTenantsFile, false, false, logger);
+            }
+
+            logger.info(String.format("Users/Groups file loaded at %s", new Date().toString()));
+        } catch (IOException | AuthorizerCreationException | JAXBException | IllegalStateException | SAXException e) {
+            throw new AuthorizerCreationException(e);
+        }
+    }
+
+    @Override
+    public Set<User> getUsers() throws AuthorizationAccessException {
+        return userGroupHolder.get().getAllUsers();
+    }
+
+    @Override
+    public synchronized User addUser(User user) throws AuthorizationAccessException {
+        if (user == null) {
+            throw new IllegalArgumentException("User cannot be null");
+        }
+
+        final org.apache.nifi.registry.security.authorization.file.tenants.generated.User jaxbUser = createJAXBUser(user);
+
+        final UserGroupHolder holder = userGroupHolder.get();
+        final Tenants tenants = holder.getTenants();
+        tenants.getUsers().getUser().add(jaxbUser);
+
+        saveAndRefreshHolder(tenants);
+
+        return userGroupHolder.get().getUsersById().get(user.getIdentifier());
+    }
+
+    @Override
+    public User getUser(String identifier) throws AuthorizationAccessException {
+        if (identifier == null) {
+            return null;
+        }
+
+        final UserGroupHolder holder = userGroupHolder.get();
+        return holder.getUsersById().get(identifier);
+    }
+
+    @Override
+    public synchronized User updateUser(User user) throws AuthorizationAccessException {
+        if (user == null) {
+            throw new IllegalArgumentException("User cannot be null");
+        }
+
+        final UserGroupHolder holder = userGroupHolder.get();
+        final Tenants tenants = holder.getTenants();
+
+        final List<org.apache.nifi.registry.security.authorization.file.tenants.generated.User> users = tenants.getUsers().getUser();
+
+        // fine the User that needs to be updated
+        org.apache.nifi.registry.security.authorization.file.tenants.generated.User updateUser = null;
+        for (org.apache.nifi.registry.security.authorization.file.tenants.generated.User jaxbUser : users) {
+            if (user.getIdentifier().equals(jaxbUser.getIdentifier())) {
+                updateUser = jaxbUser;
+                break;
+            }
+        }
+
+        // if user wasn't found return null, otherwise update the user and save changes
+        if (updateUser == null) {
+            return null;
+        } else {
+            updateUser.setIdentity(user.getIdentity());
+            saveAndRefreshHolder(tenants);
+
+            return userGroupHolder.get().getUsersById().get(user.getIdentifier());
+        }
+    }
+
+    @Override
+    public User getUserByIdentity(String identity) throws AuthorizationAccessException {
+        if (identity == null) {
+            return null;
+        }
+
+        final UserGroupHolder holder = userGroupHolder.get();
+        return holder.getUsersByIdentity().get(identity);
+    }
+
+    @Override
+    public synchronized User deleteUser(User user) throws AuthorizationAccessException {
+        if (user == null) {
+            throw new IllegalArgumentException("User cannot be null");
+        }
+
+        return deleteUser(user.getIdentifier());
+    }
+
+    @Override
+    public synchronized User deleteUser(String userIdentifier) throws AuthorizationAccessException {
+        if (userIdentifier == null) {
+            throw new IllegalArgumentException("User identifier cannot be null");
+        }
+
+        final UserGroupHolder holder = userGroupHolder.get();
+        final User deletedUser = holder.getUsersById().get(userIdentifier);
+        if (deletedUser == null) {
+            return null;
+        }
+
+        // for each group iterate over the user references and remove the user reference if it matches the user being deleted
+        final Tenants tenants = holder.getTenants();
+        for (org.apache.nifi.registry.security.authorization.file.tenants.generated.Group group : tenants.getGroups().getGroup()) {
+            Iterator<org.apache.nifi.registry.security.authorization.file.tenants.generated.Group.User> groupUserIter = group.getUser().iterator();
+            while (groupUserIter.hasNext()) {
+                org.apache.nifi.registry.security.authorization.file.tenants.generated.Group.User groupUser = groupUserIter.next();
+                if (groupUser.getIdentifier().equals(userIdentifier)) {
+                    groupUserIter.remove();
+                    break;
+                }
+            }
+        }
+
+        // remove the actual user
+        Iterator<org.apache.nifi.registry.security.authorization.file.tenants.generated.User> iter = tenants.getUsers().getUser().iterator();
+        while (iter.hasNext()) {
+            org.apache.nifi.registry.security.authorization.file.tenants.generated.User jaxbUser = iter.next();
+            if (userIdentifier.equals(jaxbUser.getIdentifier())) {
+                iter.remove();
+                break;
+            }
+        }
+
+        saveAndRefreshHolder(tenants);
+        return deletedUser;
+    }
+
+    @Override
+    public Set<Group> getGroups() throws AuthorizationAccessException {
+        return userGroupHolder.get().getAllGroups();
+    }
+
+    @Override
+    public synchronized Group addGroup(Group group) throws AuthorizationAccessException {
+        if (group == null) {
+            throw new IllegalArgumentException("Group cannot be null");
+        }
+
+        final UserGroupHolder holder = userGroupHolder.get();
+        final Tenants tenants = holder.getTenants();
+
+        // determine that all users in the group exist before doing anything, throw an exception if they don't
+        checkGroupUsers(group, tenants.getUsers().getUser());
+
+        // create a new JAXB Group based on the incoming Group
+        final org.apache.nifi.registry.security.authorization.file.tenants.generated.Group jaxbGroup =
+                new org.apache.nifi.registry.security.authorization.file.tenants.generated.Group();
+        jaxbGroup.setIdentifier(group.getIdentifier());
+        jaxbGroup.setName(group.getName());
+
+        // add each user to the group
+        for (String groupUser : group.getUsers()) {
+            org.apache.nifi.registry.security.authorization.file.tenants.generated.Group.User jaxbGroupUser =
+                    new org.apache.nifi.registry.security.authorization.file.tenants.generated.Group.User();
+            jaxbGroupUser.setIdentifier(groupUser);
+            jaxbGroup.getUser().add(jaxbGroupUser);
+        }
+
+        tenants.getGroups().getGroup().add(jaxbGroup);
+        saveAndRefreshHolder(tenants);
+
+        return userGroupHolder.get().getGroupsById().get(group.getIdentifier());
+    }
+
+    @Override
+    public Group getGroup(String identifier) throws AuthorizationAccessException {
+        if (identifier == null) {
+            return null;
+        }
+        return userGroupHolder.get().getGroupsById().get(identifier);
+    }
+
+    @Override
+    public UserAndGroups getUserAndGroups(final String identity) throws AuthorizationAccessException {
+        final UserGroupHolder holder = userGroupHolder.get();
+        final User user = holder.getUser(identity);
+        final Set<Group> groups = holder.getGroups(identity);
+
+        return new UserAndGroups() {
+            @Override
+            public User getUser() {
+                return user;
+            }
+
+            @Override
+            public Set<Group> getGroups() {
+                return groups;
+            }
+        };
+    }
+
+    @Override
+    public synchronized Group updateGroup(Group group) throws AuthorizationAccessException {
+        if (group == null) {
+            throw new IllegalArgumentException("Group cannot be null");
+        }
+
+        final UserGroupHolder holder = userGroupHolder.get();
+        final Tenants tenants = holder.getTenants();
+
+        // find the group that needs to be update
+        org.apache.nifi.registry.security.authorization.file.tenants.generated.Group updateGroup = null;
+        for (org.apache.nifi.registry.security.authorization.file.tenants.generated.Group jaxbGroup : tenants.getGroups().getGroup()) {
+            if (jaxbGroup.getIdentifier().equals(group.getIdentifier())) {
+                updateGroup = jaxbGroup;
+                break;
+            }
+        }
+
+        // if the group wasn't found return null, otherwise update the group and save changes
+        if (updateGroup == null) {
+            return null;
+        }
+
+        // reset the list of users and add each user to the group
+        updateGroup.getUser().clear();
+        for (String groupUser : group.getUsers()) {
+            org.apache.nifi.registry.security.authorization.file.tenants.generated.Group.User jaxbGroupUser =
+                    new org.apache.nifi.registry.security.authorization.file.tenants.generated.Group.User();
+            jaxbGroupUser.setIdentifier(groupUser);
+            updateGroup.getUser().add(jaxbGroupUser);
+        }
+
+        updateGroup.setName(group.getName());
+        saveAndRefreshHolder(tenants);
+
+        return userGroupHolder.get().getGroupsById().get(group.getIdentifier());
+    }
+
+    @Override
+    public synchronized Group deleteGroup(Group group) throws AuthorizationAccessException {
+        if (group == null) {
+            throw new IllegalArgumentException("Group cannot be null");
+        }
+
+        return deleteGroup(group.getIdentifier());
+    }
+
+    @Override
+    public synchronized Group deleteGroup(String groupIdentifier) throws AuthorizationAccessException {
+        if (groupIdentifier == null) {
+            throw new IllegalArgumentException("Group identifier cannot be null");
+        }
+
+        final UserGroupHolder holder = userGroupHolder.get();
+        final Group deletedGroup = holder.getGroupsById().get(groupIdentifier);
+        if (deletedGroup == null) {
+            return null;
+        }
+
+        // now remove the actual group from the top-level list of groups
+        final Tenants tenants = holder.getTenants();
+        Iterator<org.apache.nifi.registry.security.authorization.file.tenants.generated.Group> iter = tenants.getGroups().getGroup().iterator();
+        while (iter.hasNext()) {
+            org.apache.nifi.registry.security.authorization.file.tenants.generated.Group jaxbGroup = iter.next();
+            if (groupIdentifier.equals(jaxbGroup.getIdentifier())) {
+                iter.remove();
+                break;
+            }
+        }
+
+        saveAndRefreshHolder(tenants);
+        return deletedGroup;
+    }
+
+    UserGroupHolder getUserGroupHolder() {
+        return userGroupHolder.get();
+    }
+
+    @AuthorizerContext
+    public void setNiFiProperties(NiFiRegistryProperties properties) {
+        this.properties = properties;
+    }
+
+    @Override
+    public synchronized void inheritFingerprint(String fingerprint) throws AuthorizationAccessException {
+        final UsersAndGroups usersAndGroups = parseUsersAndGroups(fingerprint);
+        usersAndGroups.getUsers().forEach(user -> addUser(user));
+        usersAndGroups.getGroups().forEach(group -> addGroup(group));
+    }
+
+    @Override
+    public void checkInheritability(String proposedFingerprint) throws AuthorizationAccessException {
+        try {
+            // ensure we understand the proposed fingerprint
+            parseUsersAndGroups(proposedFingerprint);
+        } catch (final AuthorizationAccessException e) {
+            throw new UninheritableAuthorizationsException("Unable to parse the proposed fingerprint: " + e);
+        }
+
+        final UserGroupHolder usersAndGroups = userGroupHolder.get();
+
+        // ensure we are in a proper state to inherit the fingerprint
+        if (!usersAndGroups.getAllUsers().isEmpty() || !usersAndGroups.getAllGroups().isEmpty()) {
+            throw new UninheritableAuthorizationsException("Proposed fingerprint is not inheritable because the current users and groups is not empty.");
+        }
+    }
+
+    @Override
+    public String getFingerprint() throws AuthorizationAccessException {
+        final UserGroupHolder usersAndGroups = userGroupHolder.get();
+
+        final List<User> users = new ArrayList<>(usersAndGroups.getAllUsers());
+        Collections.sort(users, Comparator.comparing(User::getIdentifier));
+
+        final List<Group> groups = new ArrayList<>(usersAndGroups.getAllGroups());
+        Collections.sort(groups, Comparator.comparing(Group::getIdentifier));
+
+        XMLStreamWriter writer = null;
+        final StringWriter out = new StringWriter();
+        try {
+            writer = XML_OUTPUT_FACTORY.createXMLStreamWriter(out);
+            writer.writeStartDocument();
+            writer.writeStartElement("tenants");
+
+            for (User user : users) {
+                writeUser(writer, user);
+            }
+            for (Group group : groups) {
+                writeGroup(writer, group);
+            }
+
+            writer.writeEndElement();
+            writer.writeEndDocument();
+            writer.flush();
+        } catch (XMLStreamException e) {
+            throw new AuthorizationAccessException("Unable to generate fingerprint", e);
+        } finally {
+            if (writer != null) {
+                try {
+                    writer.close();
+                } catch (XMLStreamException e) {
+                    // nothing to do here
+                }
+            }
+        }
+
+        return out.toString();
+    }
+
+    private UsersAndGroups parseUsersAndGroups(final String fingerprint) {
+        final List<User> users = new ArrayList<>();
+        final List<Group> groups = new ArrayList<>();
+
+        final byte[] fingerprintBytes = fingerprint.getBytes(StandardCharsets.UTF_8);
+        try (final ByteArrayInputStream in = new ByteArrayInputStream(fingerprintBytes)) {
+            final DocumentBuilder docBuilder = DOCUMENT_BUILDER_FACTORY.newDocumentBuilder();
+            final Document document = docBuilder.parse(in);
+            final Element rootElement = document.getDocumentElement();
+
+            // parse all the users and add them to the current user group provider
+            NodeList userNodes = rootElement.getElementsByTagName(USER_ELEMENT);
+            for (int i=0; i < userNodes.getLength(); i++) {
+                Node userNode = userNodes.item(i);
+                users.add(parseUser((Element) userNode));
+            }
+
+            // parse all the groups and add them to the current user group provider
+            NodeList groupNodes = rootElement.getElementsByTagName(GROUP_ELEMENT);
+            for (int i=0; i < groupNodes.getLength(); i++) {
+                Node groupNode = groupNodes.item(i);
+                groups.add(parseGroup((Element) groupNode));
+            }
+        } catch (SAXException | ParserConfigurationException | IOException e) {
+            throw new AuthorizationAccessException("Unable to parse fingerprint", e);
+        }
+
+        return new UsersAndGroups(users, groups);
+    }
+
+    private User parseUser(final Element element) {
+        final User.Builder builder = new User.Builder()
+                .identifier(element.getAttribute(IDENTIFIER_ATTR))
+                .identity(element.getAttribute(IDENTITY_ATTR));
+
+        return builder.build();
+    }
+
+    private Group parseGroup(final Element element) {
+        final Group.Builder builder = new Group.Builder()
+                .identifier(element.getAttribute(IDENTIFIER_ATTR))
+                .name(element.getAttribute(NAME_ATTR));
+
+        NodeList groupUsers = element.getElementsByTagName(GROUP_USER_ELEMENT);
+        for (int i=0; i < groupUsers.getLength(); i++) {
+            Element groupUserNode = (Element) groupUsers.item(i);
+            builder.addUser(groupUserNode.getAttribute(IDENTIFIER_ATTR));
+        }
+
+        return builder.build();
+    }
+
+    private void writeUser(final XMLStreamWriter writer, final User user) throws XMLStreamException {
+        writer.writeStartElement(USER_ELEMENT);
+        writer.writeAttribute(IDENTIFIER_ATTR, user.getIdentifier());
+        writer.writeAttribute(IDENTITY_ATTR, user.getIdentity());
+        writer.writeEndElement();
+    }
+
+    private void writeGroup(final XMLStreamWriter writer, final Group group) throws XMLStreamException {
+        List<String> users = new ArrayList<>(group.getUsers());
+        Collections.sort(users);
+
+        writer.writeStartElement(GROUP_ELEMENT);
+        writer.writeAttribute(IDENTIFIER_ATTR, group.getIdentifier());
+        writer.writeAttribute(NAME_ATTR, group.getName());
+
+        for (String user : users) {
+            writer.writeStartElement(GROUP_USER_ELEMENT);
+            writer.writeAttribute(IDENTIFIER_ATTR, user);
+            writer.writeEndElement();
+        }
+
+        writer.writeEndElement();
+    }
+
+    private org.apache.nifi.registry.security.authorization.file.tenants.generated.User createJAXBUser(User user) {
+        final org.apache.nifi.registry.security.authorization.file.tenants.generated.User jaxbUser =
+                new org.apache.nifi.registry.security.authorization.file.tenants.generated.User();
+        jaxbUser.setIdentifier(user.getIdentifier());
+        jaxbUser.setIdentity(user.getIdentity());
+        return jaxbUser;
+    }
+
+    private Set<org.apache.nifi.registry.security.authorization.file.tenants.generated.User> checkGroupUsers(
+            final Group group,
+            final List<org.apache.nifi.registry.security.authorization.file.tenants.generated.User> users) {
+        final Set<org.apache.nifi.registry.security.authorization.file.tenants.generated.User> jaxbUsers = new HashSet<>();
+        for (String groupUser : group.getUsers()) {
+            boolean found = false;
+            for (org.apache.nifi.registry.security.authorization.file.tenants.generated.User jaxbUser : users) {
+                if (jaxbUser.getIdentifier().equals(groupUser)) {
+                    jaxbUsers.add(jaxbUser);
+                    found = true;
+                    break;
+                }
+            }
+
+            if (!found) {
+                throw new IllegalStateException("Unable to add group because user " + groupUser + " does not exist");
+            }
+        }
+        return jaxbUsers;
+    }
+
+    /**
+     * Loads the authorizations file and populates the AuthorizationsHolder, only called during start-up.
+     *
+     * @throws JAXBException            Unable to reload the authorized users file
+     * @throws IllegalStateException    Unable to sync file with restore
+     * @throws SAXException             Unable to unmarshall tenants
+     */
+    private synchronized void load() throws JAXBException, IllegalStateException, SAXException {
+        final Tenants tenants = unmarshallTenants();
+        if (tenants.getUsers() == null) {
+            tenants.setUsers(new Users());
+        }
+        if (tenants.getGroups() == null) {
+            tenants.setGroups(new Groups());
+        }
+
+        final UserGroupHolder userGroupHolder = new UserGroupHolder(tenants);
+        final boolean emptyTenants = userGroupHolder.getAllUsers().isEmpty() && userGroupHolder.getAllGroups().isEmpty();
+//        final boolean hasLegacyAuthorizedUsers = (legacyAuthorizedUsersFile != null && !StringUtils.isBlank(legacyAuthorizedUsersFile));
+
+        if (emptyTenants) {
+//            if (hasLegacyAuthorizedUsers) {
+//                logger.info("Loading users from legacy model " + legacyAuthorizedUsersFile + " into new users file.");
+//                convertLegacyAuthorizedUsers(tenants);
+//            }
+
+            populateInitialUsers(tenants);
+
+            // save any changes that were made and repopulate the holder
+            saveAndRefreshHolder(tenants);
+        } else {
+            this.userGroupHolder.set(userGroupHolder);
+        }
+    }
+
+    private void saveTenants(final Tenants tenants) throws JAXBException {
+        final Marshaller marshaller = JAXB_TENANTS_CONTEXT.createMarshaller();
+        marshaller.setSchema(tenantsSchema);
+        marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
+        marshaller.marshal(tenants, tenantsFile);
+    }
+
+    private Tenants unmarshallTenants() throws JAXBException {
+        final Unmarshaller unmarshaller = JAXB_TENANTS_CONTEXT.createUnmarshaller();
+        unmarshaller.setSchema(tenantsSchema);
+
+        final JAXBElement<Tenants> element = unmarshaller.unmarshal(new StreamSource(tenantsFile), Tenants.class);
+        return element.getValue();
+    }
+
+    private void populateInitialUsers(final Tenants tenants) {
+        for (String initialUserIdentity : initialUserIdentities) {
+            getOrCreateUser(tenants, initialUserIdentity);
+        }
+    }
+
+    /**
+     * Finds the User with the given identity, or creates a new one and adds it to the Tenants.
+     *
+     * @param tenants the Tenants reference
+     * @param userIdentity the user identity to find or create
+     * @return the User from Tenants with the given identity, or a new instance that was added to Tenants
+     */
+    private org.apache.nifi.registry.security.authorization.file.tenants.generated.User getOrCreateUser(final Tenants tenants, final String userIdentity) {
+        if (StringUtils.isBlank(userIdentity)) {
+            return null;
+        }
+
+        org.apache.nifi.registry.security.authorization.file.tenants.generated.User foundUser = null;
+        for (org.apache.nifi.registry.security.authorization.file.tenants.generated.User user : tenants.getUsers().getUser()) {
+            if (user.getIdentity().equals(userIdentity)) {
+                foundUser = user;
+                break;
+            }
+        }
+
+        if (foundUser == null) {
+            final String userIdentifier = IdentifierUtil.getIdentifier(userIdentity);
+            foundUser = new org.apache.nifi.registry.security.authorization.file.tenants.generated.User();
+            foundUser.setIdentifier(userIdentifier);
+            foundUser.setIdentity(userIdentity);
+            tenants.getUsers().getUser().add(foundUser);
+        }
+
+        return foundUser;
+    }
+
+    /**
+     * Finds the Group with the given name, or creates a new one and adds it to Tenants.
+     *
+     * @param tenants the Tenants reference
+     * @param groupName the name of the group to look for
+     * @return the Group from Tenants with the given name, or a new instance that was added to Tenants
+     */
+    private org.apache.nifi.registry.security.authorization.file.tenants.generated.Group getOrCreateGroup(final Tenants tenants, final String groupName) {
+        if (StringUtils.isBlank(groupName)) {
+            return null;
+        }
+
+        org.apache.nifi.registry.security.authorization.file.tenants.generated.Group foundGroup = null;
+        for (org.apache.nifi.registry.security.authorization.file.tenants.generated.Group group : tenants.getGroups().getGroup()) {
+            if (group.getName().equals(groupName)) {
+                foundGroup = group;
+                break;
+            }
+        }
+
+        if (foundGroup == null) {
+            final String newGroupIdentifier = IdentifierUtil.getIdentifier(groupName);
+            foundGroup = new org.apache.nifi.registry.security.authorization.file.tenants.generated.Group();
+            foundGroup.setIdentifier(newGroupIdentifier);
+            foundGroup.setName(groupName);
+            tenants.getGroups().getGroup().add(foundGroup);
+        }
+
+        return foundGroup;
+    }
+
+    /**
+     * Saves the Authorizations instance by marshalling to a file, then re-populates the
+     * in-memory data structures and sets the new holder.
+     *
+     * Synchronized to ensure only one thread writes the file at a time.
+     *
+     * @param tenants the tenants to save and populate from
+     * @throws AuthorizationAccessException if an error occurs saving the authorizations
+     */
+    private synchronized void saveAndRefreshHolder(final Tenants tenants) throws AuthorizationAccessException {
+        try {
+            saveTenants(tenants);
+
+            this.userGroupHolder.set(new UserGroupHolder(tenants));
+        } catch (JAXBException e) {
+            throw new AuthorizationAccessException("Unable to save Authorizations", e);
+        }
+    }
+
+    @Override
+    public void preDestruction() throws AuthorizerDestructionException {
+    }
+
+    private static class UsersAndGroups {
+        final List<User> users;
+        final List<Group> groups;
+
+        public UsersAndGroups(List<User> users, List<Group> groups) {
+            this.users = users;
+            this.groups = groups;
+        }
+
+        public List<User> getUsers() {
+            return users;
+        }
+
+        public List<Group> getGroups() {
+            return groups;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/IdentifierUtil.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/IdentifierUtil.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/IdentifierUtil.java
new file mode 100644
index 0000000..91e673f
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/IdentifierUtil.java
@@ -0,0 +1,35 @@
+/*
+ * 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.authorization.file;
+
+import org.apache.commons.lang3.StringUtils;
+
+import java.nio.charset.StandardCharsets;
+import java.util.UUID;
+
+public final class IdentifierUtil {
+
+    static String getIdentifier(final String seed) {
+        if (StringUtils.isBlank(seed)) {
+            return null;
+        }
+
+        return UUID.nameUUIDFromBytes(seed.getBytes(StandardCharsets.UTF_8)).toString();
+    }
+
+    private IdentifierUtil() {}
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/UserGroupHolder.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/UserGroupHolder.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/UserGroupHolder.java
new file mode 100644
index 0000000..9828a45
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/file/UserGroupHolder.java
@@ -0,0 +1,241 @@
+/*
+ * 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.authorization.file;
+
+
+import org.apache.nifi.registry.security.authorization.file.tenants.generated.Groups;
+import org.apache.nifi.registry.security.authorization.file.tenants.generated.Tenants;
+import org.apache.nifi.registry.security.authorization.file.tenants.generated.Users;
+import org.apache.nifi.registry.security.authorization.Group;
+import org.apache.nifi.registry.security.authorization.User;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A holder to provide atomic access to user group data structures.
+ */
+public class UserGroupHolder {
+
+    private final Tenants tenants;
+
+    private final Set<User> allUsers;
+    private final Map<String,User> usersById;
+    private final Map<String,User> usersByIdentity;
+
+    private final Set<Group> allGroups;
+    private final Map<String,Group> groupsById;
+    private final Map<String, Set<Group>> groupsByUserIdentity;
+
+    /**
+     * Creates a new holder and populates all convenience data structures.
+     *
+     * @param tenants the current tenants instance
+     */
+    public UserGroupHolder(final Tenants tenants) {
+        this.tenants = tenants;
+
+        // load all users
+        final Users users = tenants.getUsers();
+        final Set<User> allUsers = Collections.unmodifiableSet(createUsers(users));
+
+        // load all groups
+        final Groups groups = tenants.getGroups();
+        final Set<Group> allGroups = Collections.unmodifiableSet(createGroups(groups, users));
+
+        // create a convenience map to retrieve a user by id
+        final Map<String, User> userByIdMap = Collections.unmodifiableMap(createUserByIdMap(allUsers));
+
+        // create a convenience map to retrieve a user by identity
+        final Map<String, User> userByIdentityMap = Collections.unmodifiableMap(createUserByIdentityMap(allUsers));
+
+        // create a convenience map to retrieve a group by id
+        final Map<String, Group> groupByIdMap = Collections.unmodifiableMap(createGroupByIdMap(allGroups));
+
+        // create a convenience map to retrieve the groups for a user identity
+        final Map<String, Set<Group>> groupsByUserIdentityMap = Collections.unmodifiableMap(createGroupsByUserIdentityMap(allGroups, allUsers));
+
+        // set all the holders
+        this.allUsers = allUsers;
+        this.allGroups = allGroups;
+        this.usersById = userByIdMap;
+        this.usersByIdentity = userByIdentityMap;
+        this.groupsById = groupByIdMap;
+        this.groupsByUserIdentity = groupsByUserIdentityMap;
+    }
+
+    /**
+     * Creates a set of Users from the JAXB Users.
+     *
+     * @param users the JAXB Users
+     * @return a set of API Users matching the provided JAXB Users
+     */
+    private Set<User> createUsers(Users users) {
+        Set<User> allUsers = new HashSet<>();
+        if (users == null || users.getUser() == null) {
+            return allUsers;
+        }
+
+        for (org.apache.nifi.registry.security.authorization.file.tenants.generated.User user : users.getUser()) {
+            final User.Builder builder = new User.Builder()
+                    .identity(user.getIdentity())
+                    .identifier(user.getIdentifier());
+
+            allUsers.add(builder.build());
+        }
+
+        return allUsers;
+    }
+
+    /**
+     * Creates a set of Groups from the JAXB Groups.
+     *
+     * @param groups the JAXB Groups
+     * @return a set of API Groups matching the provided JAXB Groups
+     */
+    private Set<Group> createGroups(Groups groups,
+                                    Users users) {
+        Set<Group> allGroups = new HashSet<>();
+        if (groups == null || groups.getGroup() == null) {
+            return allGroups;
+        }
+
+        for (org.apache.nifi.registry.security.authorization.file.tenants.generated.Group group : groups.getGroup()) {
+            final Group.Builder builder = new Group.Builder()
+                    .identifier(group.getIdentifier())
+                    .name(group.getName());
+
+            for (org.apache.nifi.registry.security.authorization.file.tenants.generated.Group.User groupUser : group.getUser()) {
+                builder.addUser(groupUser.getIdentifier());
+            }
+
+            allGroups.add(builder.build());
+        }
+
+        return allGroups;
+    }
+
+    /**
+     * Creates a Map from user identifier to User.
+     *
+     * @param users the set of all users
+     * @return the Map from user identifier to User
+     */
+    private Map<String,User> createUserByIdMap(final Set<User> users) {
+        Map<String,User> usersMap = new HashMap<>();
+        for (User user : users) {
+            usersMap.put(user.getIdentifier(), user);
+        }
+        return usersMap;
+    }
+
+    /**
+     * Creates a Map from user identity to User.
+     *
+     * @param users the set of all users
+     * @return the Map from user identity to User
+     */
+    private Map<String,User> createUserByIdentityMap(final Set<User> users) {
+        Map<String,User> usersMap = new HashMap<>();
+        for (User user : users) {
+            usersMap.put(user.getIdentity(), user);
+        }
+        return usersMap;
+    }
+
+    /**
+     * Creates a Map from group identifier to Group.
+     *
+     * @param groups the set of all groups
+     * @return the Map from group identifier to Group
+     */
+    private Map<String,Group> createGroupByIdMap(final Set<Group> groups) {
+        Map<String,Group> groupsMap = new HashMap<>();
+        for (Group group : groups) {
+            groupsMap.put(group.getIdentifier(), group);
+        }
+        return groupsMap;
+    }
+
+    /**
+     * Creates a Map from user identity to the set of Groups for that identity.
+     *
+     * @param groups all groups
+     * @param users all users
+     * @return a Map from User identity to the set of Groups for that identity
+     */
+    private Map<String, Set<Group>> createGroupsByUserIdentityMap(final Set<Group> groups, final Set<User> users) {
+        Map<String, Set<Group>> groupsByUserIdentity = new HashMap<>();
+
+        for (User user : users) {
+            Set<Group> userGroups = new HashSet<>();
+            for (Group group : groups) {
+                for (String groupUser : group.getUsers()) {
+                    if (groupUser.equals(user.getIdentifier())) {
+                        userGroups.add(group);
+                    }
+                }
+            }
+
+            groupsByUserIdentity.put(user.getIdentity(), userGroups);
+        }
+
+        return groupsByUserIdentity;
+    }
+
+    public Tenants getTenants() {
+        return tenants;
+    }
+
+    public Set<User> getAllUsers() {
+        return allUsers;
+    }
+
+    public Map<String, User> getUsersById() {
+        return usersById;
+    }
+
+    public Map<String, User> getUsersByIdentity() {
+        return usersByIdentity;
+    }
+
+    public Set<Group> getAllGroups() {
+        return allGroups;
+    }
+
+    public Map<String, Group> getGroupsById() {
+        return groupsById;
+    }
+
+    public User getUser(String identity) {
+        if (identity == null) {
+            throw new IllegalArgumentException("Identity cannot be null");
+        }
+        return usersByIdentity.get(identity);
+    }
+
+    public Set<Group> getGroups(String userIdentity) {
+        if (userIdentity == null) {
+            throw new IllegalArgumentException("User Identity cannot be null");
+        }
+        return groupsByUserIdentity.get(userIdentity);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/AccessPolicyAuthorizable.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/AccessPolicyAuthorizable.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/AccessPolicyAuthorizable.java
new file mode 100644
index 0000000..de28565
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/AccessPolicyAuthorizable.java
@@ -0,0 +1,122 @@
+/*
+ * 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.authorization.resource;
+
+import org.apache.nifi.registry.security.authorization.exception.AccessDeniedException;
+import org.apache.nifi.registry.security.authorization.AuthorizationResult;
+import org.apache.nifi.registry.security.authorization.AuthorizationResult.Result;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.Resource;
+import org.apache.nifi.registry.security.authorization.user.NiFiUser;
+
+import java.util.Map;
+
+/**
+ * Authorizable for policies of an Authorizable.
+ */
+public class AccessPolicyAuthorizable implements Authorizable, EnforcePolicyPermissionsThroughBaseResource {
+
+    private static final Authorizable POLICIES_AUTHORIZABLE = new Authorizable() {
+        @Override
+        public Authorizable getParentAuthorizable() {
+            return null;
+        }
+
+        @Override
+        public Resource getResource() {
+            return ResourceFactory.getPoliciesResource();
+        }
+    };
+
+    final Authorizable authorizable;
+
+    public AccessPolicyAuthorizable(Authorizable authorizable) {
+        this.authorizable = authorizable;
+    }
+
+    @Override
+    public Authorizable getBaseAuthorizable() {
+        return authorizable;
+    }
+
+    @Override
+    public Authorizable getParentAuthorizable() {
+        final Authorizable effectiveAuthorizable = getEffectiveAuthorizable();
+        if (effectiveAuthorizable.getParentAuthorizable() == null) {
+            return POLICIES_AUTHORIZABLE;
+        } else {
+            return new AccessPolicyAuthorizable(effectiveAuthorizable.getParentAuthorizable());
+        }
+    }
+
+    @Override
+    public Resource getResource() {
+        return ResourceFactory.getPolicyResource(getEffectiveAuthorizable().getResource());
+    }
+
+    private Authorizable getEffectiveAuthorizable() {
+        // possibly consider the base resource if the authorizable uses it to enforce policy permissions
+        if (authorizable instanceof EnforcePolicyPermissionsThroughBaseResource) {
+            final Authorizable baseAuthorizable = ((EnforcePolicyPermissionsThroughBaseResource) authorizable).getBaseAuthorizable();
+
+            // if the base authorizable is for a policy, we don't want to use the base otherwise it would keep unwinding and would eventually
+            // evaluate to the policy of the component and not the policy of the policies for the component
+            if (baseAuthorizable instanceof AccessPolicyAuthorizable) {
+                return authorizable;
+            } else {
+                return baseAuthorizable;
+            }
+        } else {
+            return authorizable;
+        }
+    }
+
+    @Override
+    public AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) {
+        if (user == null) {
+            throw new AccessDeniedException("Unknown user.");
+        }
+
+        final AuthorizationResult resourceResult = Authorizable.super.checkAuthorization(authorizer, action, user, resourceContext);
+
+        // if we're denied from the resource try inheriting
+        if (Result.Denied.equals(resourceResult.getResult())) {
+            return getParentAuthorizable().checkAuthorization(authorizer, action, user, resourceContext);
+        } else {
+            return resourceResult;
+        }
+    }
+
+    @Override
+    public void authorize(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) throws AccessDeniedException {
+        if (user == null) {
+            throw new AccessDeniedException("Unknown user.");
+        }
+
+        try {
+            Authorizable.super.authorize(authorizer, action, user, resourceContext);
+        } catch (final AccessDeniedException resourceDenied) {
+            // if we're denied from the resource try inheriting
+            try {
+                getParentAuthorizable().authorize(authorizer, action, user, resourceContext);
+            } catch (final AccessDeniedException policiesDenied) {
+                throw resourceDenied;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/Authorizable.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/Authorizable.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/Authorizable.java
new file mode 100644
index 0000000..d08467e
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/Authorizable.java
@@ -0,0 +1,300 @@
+/*
+ * 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.authorization.resource;
+
+import org.apache.nifi.registry.security.authorization.AuthorizationResult.Result;
+import org.apache.nifi.registry.security.authorization.exception.AccessDeniedException;
+import org.apache.nifi.registry.security.authorization.user.NiFiUser;
+import org.apache.nifi.registry.security.authorization.AuthorizationAuditor;
+import org.apache.nifi.registry.security.authorization.AuthorizationRequest;
+import org.apache.nifi.registry.security.authorization.AuthorizationResult;
+import org.apache.nifi.registry.security.authorization.Authorizer;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.Resource;
+import org.apache.nifi.registry.security.authorization.UserContextKeys;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public interface Authorizable {
+
+    /**
+     * The parent for this Authorizable. May be null.
+     *
+     * @return the parent authorizable or null
+     */
+    Authorizable getParentAuthorizable();
+
+    /**
+     * The Resource for this Authorizable.
+     *
+     * @return the resource
+     */
+    Resource getResource();
+
+    /**
+     * The originally requested resource for this Authorizable. Because policies are inherited, if a resource
+     * does not have a policy, this Authorizable may represent a parent resource and this method will return
+     * the originally requested resource.
+     *
+     * @return the originally requested resource
+     */
+    default Resource getRequestedResource() {
+        return getResource();
+    }
+
+    /**
+     * Returns whether the current user is authorized for the specified action on the specified resource. This
+     * method does not imply the user is directly attempting to access the specified resource. If the user is
+     * attempting a direct access use Authorizable.authorize().
+     *
+     * @param authorizer authorizer
+     * @param action action
+     * @return is authorized
+     */
+    default boolean isAuthorized(Authorizer authorizer, RequestAction action, NiFiUser user) {
+        return Result.Approved.equals(checkAuthorization(authorizer, action, user).getResult());
+    }
+
+    /**
+     * Returns the result of an authorization request for the specified user for the specified action on the specified
+     * resource. This method does not imply the user is directly attempting to access the specified resource. If the user is
+     * attempting a direct access use Authorizable.authorize().
+     *
+     * @param authorizer authorizer
+     * @param action action
+     * @param user user
+     * @return is authorized
+     */
+    default AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) {
+        if (user == null) {
+            return AuthorizationResult.denied("Unknown user.");
+        }
+
+        final Map<String,String> userContext;
+        if (user.getClientAddress() != null && !user.getClientAddress().trim().isEmpty()) {
+            userContext = new HashMap<>();
+            userContext.put(UserContextKeys.CLIENT_ADDRESS.name(), user.getClientAddress());
+        } else {
+            userContext = null;
+        }
+
+        final Resource resource = getResource();
+        final Resource requestedResource = getRequestedResource();
+        final AuthorizationRequest request = new AuthorizationRequest.Builder()
+                .identity(user.getIdentity())
+                .groups(user.getGroups())
+                .anonymous(user.isAnonymous())
+                .accessAttempt(false)
+                .action(action)
+                .resource(resource)
+                .requestedResource(requestedResource)
+                .resourceContext(resourceContext)
+                .userContext(userContext)
+                .explanationSupplier(() -> {
+                    // build the safe explanation
+                    final StringBuilder safeDescription = new StringBuilder("Unable to ");
+
+                    if (RequestAction.READ.equals(action)) {
+                        safeDescription.append("view ");
+                    } else {
+                        safeDescription.append("modify "); // covers write or delete
+                    }
+                    safeDescription.append(resource.getSafeDescription()).append(".");
+
+                    return safeDescription.toString();
+                })
+                .build();
+
+        // perform the authorization
+        final AuthorizationResult result = authorizer.authorize(request);
+
+        // verify the results
+        if (Result.ResourceNotFound.equals(result.getResult())) {
+            final Authorizable parent = getParentAuthorizable();
+            if (parent == null) {
+                return AuthorizationResult.denied("No applicable policies could be found.");
+            } else {
+                // create a custom authorizable to override the safe description but still defer to the parent authorizable
+                final Authorizable parentProxy = new Authorizable() {
+                    @Override
+                    public Authorizable getParentAuthorizable() {
+                        return parent.getParentAuthorizable();
+                    }
+
+                    @Override
+                    public Resource getRequestedResource() {
+                        return requestedResource;
+                    }
+
+                    @Override
+                    public Resource getResource() {
+                        final Resource parentResource = parent.getResource();
+                        return new Resource() {
+                            @Override
+                            public String getIdentifier() {
+                                return parentResource.getIdentifier();
+                            }
+
+                            @Override
+                            public String getName() {
+                                return parentResource.getName();
+                            }
+
+                            @Override
+                            public String getSafeDescription() {
+                                return resource.getSafeDescription();
+                            }
+                        };
+                    }
+                };
+                return parentProxy.checkAuthorization(authorizer, action, user, resourceContext);
+            }
+        } else {
+            return result;
+        }
+    }
+
+    /**
+     * Returns the result of an authorization request for the specified user for the specified action on the specified
+     * resource. This method does not imply the user is directly attempting to access the specified resource. If the user is
+     * attempting a direct access use Authorizable.authorize().
+     *
+     * @param authorizer authorizer
+     * @param action action
+     * @param user user
+     * @return is authorized
+     */
+    default AuthorizationResult checkAuthorization(Authorizer authorizer, RequestAction action, NiFiUser user) {
+        return checkAuthorization(authorizer, action, user, null);
+    }
+
+    /**
+     * Authorizes the current user for the specified action on the specified resource. This method does imply the user is
+     * directly accessing the specified resource.
+     *
+     * @param authorizer authorizer
+     * @param action action
+     * @param user user
+     * @param resourceContext resource context
+     */
+    default void authorize(Authorizer authorizer, RequestAction action, NiFiUser user, Map<String, String> resourceContext) throws AccessDeniedException {
+        if (user == null) {
+            throw new AccessDeniedException("Unknown user.");
+        }
+
+        final Map<String,String> userContext;
+        if (user.getClientAddress() != null && !user.getClientAddress().trim().isEmpty()) {
+            userContext = new HashMap<>();
+            userContext.put(UserContextKeys.CLIENT_ADDRESS.name(), user.getClientAddress());
+        } else {
+            userContext = null;
+        }
+
+        final Resource resource = getResource();
+        final Resource requestedResource = getRequestedResource();
+        final AuthorizationRequest request = new AuthorizationRequest.Builder()
+                .identity(user.getIdentity())
+                .groups(user.getGroups())
+                .anonymous(user.isAnonymous())
+                .accessAttempt(true)
+                .action(action)
+                .resource(resource)
+                .requestedResource(requestedResource)
+                .resourceContext(resourceContext)
+                .userContext(userContext)
+                .explanationSupplier(() -> {
+                    // build the safe explanation
+                    final StringBuilder safeDescription = new StringBuilder("Unable to ");
+
+                    if (RequestAction.READ.equals(action)) {
+                        safeDescription.append("view ");
+                    } else {
+                        safeDescription.append("modify ");
+                    }
+                    safeDescription.append(resource.getSafeDescription()).append(".");
+
+                    return safeDescription.toString();
+                })
+                .build();
+
+        final AuthorizationResult result = authorizer.authorize(request);
+        if (Result.ResourceNotFound.equals(result.getResult())) {
+            final Authorizable parent = getParentAuthorizable();
+            if (parent == null) {
+                final AuthorizationResult failure = AuthorizationResult.denied("No applicable policies could be found.");
+
+                // audit authorization request
+                if (authorizer instanceof AuthorizationAuditor) {
+                    ((AuthorizationAuditor) authorizer).auditAccessAttempt(request, failure);
+                }
+
+                // denied
+                throw new AccessDeniedException(failure.getExplanation());
+            } else {
+                // create a custom authorizable to override the safe description but still defer to the parent authorizable
+                final Authorizable parentProxy = new Authorizable() {
+                    @Override
+                    public Authorizable getParentAuthorizable() {
+                        return parent.getParentAuthorizable();
+                    }
+
+                    @Override
+                    public Resource getRequestedResource() {
+                        return requestedResource;
+                    }
+
+                    @Override
+                    public Resource getResource() {
+                        final Resource parentResource = parent.getResource();
+                        return new Resource() {
+                            @Override
+                            public String getIdentifier() {
+                                return parentResource.getIdentifier();
+                            }
+
+                            @Override
+                            public String getName() {
+                                return parentResource.getName();
+                            }
+
+                            @Override
+                            public String getSafeDescription() {
+                                return resource.getSafeDescription();
+                            }
+                        };
+                    }
+                };
+                parentProxy.authorize(authorizer, action, user, resourceContext);
+            }
+        } else if (Result.Denied.equals(result.getResult())) {
+            throw new AccessDeniedException(result.getExplanation());
+        }
+    }
+
+    /**
+     * Authorizes the current user for the specified action on the specified resource. This method does imply the user is
+     * directly accessing the specified resource.
+     *
+     * @param authorizer authorizer
+     * @param action action
+     * @param user user
+     */
+    default void authorize(Authorizer authorizer, RequestAction action, NiFiUser user) throws AccessDeniedException {
+        authorize(authorizer, action, user, null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/EnforcePolicyPermissionsThroughBaseResource.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/EnforcePolicyPermissionsThroughBaseResource.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/EnforcePolicyPermissionsThroughBaseResource.java
new file mode 100644
index 0000000..90fc143
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/EnforcePolicyPermissionsThroughBaseResource.java
@@ -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.
+ */
+package org.apache.nifi.registry.security.authorization.resource;
+
+/**
+ * Defers permissions on policies to the policies of the base authorizable. Required because we don't
+ * want to change the enforcement of the policies on the authorizable. For example...
+ *
+ * if a user has permissions to /policies/input-ports/1234 then they have permissions to the following
+ *
+ * - the policy for /buckets/1234                    -> /policies/buckets/1234
+ * - the policy for /policies/buckets/1234           -> /policies/policies/buckets/1234
+ */
+public interface EnforcePolicyPermissionsThroughBaseResource {
+
+    /**
+     * Returns the base authorizable. Cannot be null.
+     *
+     * @return base authorizable
+     */
+    Authorizable getBaseAuthorizable();
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/ResourceFactory.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/ResourceFactory.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/ResourceFactory.java
new file mode 100644
index 0000000..c12256c
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/ResourceFactory.java
@@ -0,0 +1,261 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.security.authorization.resource;
+
+import org.apache.nifi.registry.security.authorization.Resource;
+
+import java.util.Objects;
+
+public final class ResourceFactory {
+
+    private final static Resource BUCKETS_RESOURCE = new Resource() {
+        @Override
+        public String getIdentifier() {
+            return ResourceType.Bucket.getValue();
+        }
+
+        @Override
+        public String getName() {
+            return "Buckets";
+        }
+
+        @Override
+        public String getSafeDescription() {
+            return "buckets";
+        }
+    };
+
+
+    private final static Resource POLICY_RESOURCE = new Resource() {
+        @Override
+        public String getIdentifier() {
+            return ResourceType.Policy.getValue();
+        }
+
+        @Override
+        public String getName() {
+            return "Policies for ";
+        }
+
+        @Override
+        public String getSafeDescription() {
+            return "the policies for ";
+        }
+    };
+
+    private final static Resource PROXY_RESOURCE = new Resource() {
+        @Override
+        public String getIdentifier() {
+            return ResourceType.Proxy.getValue();
+        }
+
+        @Override
+        public String getName() {
+            return "Proxy User Requests";
+        }
+
+        @Override
+        public String getSafeDescription() {
+            return "proxy requests on behalf of users";
+        }
+    };
+
+    private final static Resource RESOURCE_RESOURCE = new Resource() {
+        @Override
+        public String getIdentifier() {
+            return ResourceType.Resource.getValue();
+        }
+
+        @Override
+        public String getName() {
+            return "NiFi Resources";
+        }
+
+        @Override
+        public String getSafeDescription() {
+            return "resources";
+        }
+    };
+
+    private final static Resource TENANT_RESOURCE = new Resource() {
+        @Override
+        public String getIdentifier() {
+            return ResourceType.Tenant.getValue();
+        }
+
+        @Override
+        public String getName() {
+            return "Tenant";
+        }
+
+        @Override
+        public String getSafeDescription() {
+            return "users/user groups";
+        }
+    };
+
+    private final static Resource POLICIES_RESOURCE = new Resource() {
+
+        @Override
+        public String getIdentifier() {
+            return "/policies";
+        }
+
+        @Override
+        public String getName() {
+            return "Access Policies";
+        }
+
+        @Override
+        public String getSafeDescription() {
+            return "policies";
+        }
+    };
+
+
+    /**
+     * Gets the Resource for proxying a user request.
+     *
+     * @return  The resource for proxying a user request
+     */
+    public static Resource getProxyResource() {
+        return PROXY_RESOURCE;
+    }
+
+    /**
+     * Gets the Resource for detailing all available NiFi Resources.
+     *
+     * @return  The Resource resource
+     */
+    public static Resource getResourceResource() {
+        return RESOURCE_RESOURCE;
+    }
+
+    /**
+     * Gets the Resource for accessing Tenants which includes creating, modifying, and deleting Users and UserGroups.
+     *
+     * @return The Resource for accessing Tenants
+     */
+    public static Resource getTenantResource() {
+        return TENANT_RESOURCE;
+    }
+
+    /**
+     * Gets the {@link Resource} for accessing access policies.
+     * @return The policies resource
+     */
+    public static Resource getPoliciesResource() {
+        return POLICIES_RESOURCE;
+    }
+
+    /**
+     * Gets the {@link Resource} for accessing buckets.
+     * @return The buckets resource
+     */
+    public static Resource getBucketsResource() {
+        return BUCKETS_RESOURCE;
+    }
+
+    /**
+     * Gets the {@link Resource} for accessing buckets.
+     * @return The buckets resource
+     */
+    public static Resource getBucketResource(String bucketIdentifier, String bucketName) {
+        return getChildResource(ResourceType.Bucket, bucketIdentifier, bucketName);
+    }
+
+    /**
+     * Gets a Resource for accessing a resources's policies.
+     *
+     * @param resource      The resource being accessed
+     * @return              The resource
+     */
+    public static Resource getPolicyResource(final Resource resource) {
+        Objects.requireNonNull(resource, "The resource type must be specified.");
+
+        return new Resource() {
+            @Override
+            public String getIdentifier() {
+                return String.format("%s%s", POLICY_RESOURCE.getIdentifier(), resource.getIdentifier());
+            }
+
+            @Override
+            public String getName() {
+                return POLICY_RESOURCE.getName() + resource.getName();
+            }
+
+            @Override
+            public String getSafeDescription() {
+                return POLICY_RESOURCE.getSafeDescription() + resource.getSafeDescription();
+            }
+        };
+    }
+
+    /**
+     * Get a Resource object for any object that has a base type and an identifier, ie:
+     * /buckets/{uuid}
+     *
+     * @param parentResourceType - Required, the base resource type
+     * @param childIdentifier - Required, the identity of this sub resource
+     * @param name - Optional, the name of the subresource
+     * @return A resource for this object
+     */
+    public static Resource getChildResource(final ResourceType parentResourceType, final String childIdentifier, final String name) {
+        Objects.requireNonNull(parentResourceType, "The base resource type must be specified.");
+        Objects.requireNonNull(childIdentifier, "The child identifier identifier must be specified.");
+
+        return new Resource() {
+            @Override
+            public String getIdentifier() {
+                return String.format("%s/%s", parentResourceType.getValue(), childIdentifier);
+            }
+
+            @Override
+            public String getName() {
+                return name;
+            }
+
+            @Override
+            public String getSafeDescription() {
+                final StringBuilder safeDescription = new StringBuilder();
+                switch (parentResourceType) {
+                    case Bucket:
+                        safeDescription.append("Bucket");
+                        break;
+                    case Policy:
+                        safeDescription.append("Policy");
+                        break;
+                    case Tenant:
+                        safeDescription.append("Tenant");
+                        break;
+                    default:
+                        safeDescription.append("Unknown resource type");
+                        break;
+                }
+                safeDescription.append(" with ID ");
+                safeDescription.append(childIdentifier);
+                return safeDescription.toString();
+            }
+        };
+
+    }
+
+    /**
+     * Prevent outside instantiation.
+     */
+    private ResourceFactory() {}
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/ResourceType.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/ResourceType.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/ResourceType.java
new file mode 100644
index 0000000..a49d973
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/resource/ResourceType.java
@@ -0,0 +1,52 @@
+/*
+ * 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.authorization.resource;
+
+public enum ResourceType {
+    Bucket("/buckets"),
+    Policy("/policies"),
+    Proxy("/proxy"),
+    Resource("/resources"),
+    Tenant("/tenants");
+
+    final String value;
+
+    private ResourceType(final String value) {
+        this.value = value;
+    }
+
+    public String getValue() {
+        return value;
+    }
+
+    public static ResourceType valueOfValue(final String rawValue) {
+        ResourceType type = null;
+
+        for (final ResourceType rt : values()) {
+            if (rt.getValue().equals(rawValue)) {
+                type = rt;
+                break;
+            }
+        }
+
+        if (type == null) {
+            throw new IllegalArgumentException("Unknown resource type value " + rawValue);
+        }
+
+        return type;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUser.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUser.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUser.java
new file mode 100644
index 0000000..47127b6
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUser.java
@@ -0,0 +1,52 @@
+/*
+ * 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.authorization.user;
+
+import java.util.Set;
+
+/**
+ * A representation of a NiFi user that has logged into the application
+ */
+public interface NiFiUser {
+
+    /**
+     * @return the unique identity of this user
+     */
+    String getIdentity();
+
+    /**
+     * @return the groups that this user belongs to if this nifi is configured to load user groups, null otherwise.
+     */
+    Set<String> getGroups();
+
+    /**
+     * @return the next user in the proxied entities chain, or <code>null</code> if no more users exist in the chain.
+     */
+    NiFiUser getChain();
+
+    /**
+     * @return <code>true</code> if the user is the unauthenticated Anonymous user
+     */
+    boolean isAnonymous();
+
+    /**
+     * @return the address of the client that made the request which created this user
+     */
+    String getClientAddress();
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUserDetails.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUserDetails.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUserDetails.java
new file mode 100644
index 0000000..ca6ea2e
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUserDetails.java
@@ -0,0 +1,91 @@
+/*
+ * 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.authorization.user;
+
+import org.apache.commons.lang3.StringUtils;
+import org.springframework.security.core.GrantedAuthority;
+import org.springframework.security.core.userdetails.UserDetails;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * User details for a NiFi user.
+ */
+public class NiFiUserDetails implements UserDetails {
+
+    private final NiFiUser user;
+
+    /**
+     * Creates a new NiFiUserDetails.
+     *
+     * @param user user
+     */
+    public NiFiUserDetails(NiFiUser user) {
+        this.user = user;
+    }
+
+    /**
+     * Get the user for this UserDetails.
+     *
+     * @return user
+     */
+    public NiFiUser getNiFiUser() {
+        return user;
+    }
+
+    /**
+     * Returns the authorities that this NiFi user has.
+     *
+     * @return authorities
+     */
+    @Override
+    public Collection<? extends GrantedAuthority> getAuthorities() {
+        return Collections.EMPTY_SET;
+    }
+
+    @Override
+    public String getPassword() {
+        return StringUtils.EMPTY;
+    }
+
+    @Override
+    public String getUsername() {
+        return user.getIdentity();
+    }
+
+    @Override
+    public boolean isAccountNonExpired() {
+        return true;
+    }
+
+    @Override
+    public boolean isAccountNonLocked() {
+        return true;
+    }
+
+    @Override
+    public boolean isCredentialsNonExpired() {
+        return true;
+    }
+
+    @Override
+    public boolean isEnabled() {
+        return true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/nifi-registry/blob/90f36dd2/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUserUtils.java
----------------------------------------------------------------------
diff --git a/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUserUtils.java b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUserUtils.java
new file mode 100644
index 0000000..b5147ea
--- /dev/null
+++ b/nifi-registry-framework/src/main/java/org/apache/nifi/registry/security/authorization/user/NiFiUserUtils.java
@@ -0,0 +1,91 @@
+/*
+ * 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.authorization.user;
+
+import org.apache.commons.lang3.StringUtils;
+import org.springframework.security.core.Authentication;
+import org.springframework.security.core.context.SecurityContext;
+import org.springframework.security.core.context.SecurityContextHolder;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Utility methods for retrieving information about the current application user.
+ *
+ */
+public final class NiFiUserUtils {
+
+    /**
+     * Returns the current NiFiUser or null if the current user is not a NiFiUser.
+     *
+     * @return user
+     */
+    public static NiFiUser getNiFiUser() {
+        NiFiUser user = null;
+
+        // obtain the principal in the current authentication
+        final SecurityContext context = SecurityContextHolder.getContext();
+        final Authentication authentication = context.getAuthentication();
+        if (authentication != null) {
+            Object principal = authentication.getPrincipal();
+            if (principal instanceof NiFiUserDetails) {
+                user = ((NiFiUserDetails) principal).getNiFiUser();
+            }
+        }
+
+        return user;
+    }
+
+    public static String getNiFiUserIdentity() {
+        // get the nifi user to extract the username
+        NiFiUser user = NiFiUserUtils.getNiFiUser();
+        if (user == null) {
+            return "unknown";
+        } else {
+            return user.getIdentity();
+        }
+    }
+
+    /**
+     * Builds the proxy chain for the specified user.
+     *
+     * @param user The current user
+     * @return The proxy chain for that user in List form
+     */
+    public static List<String> buildProxiedEntitiesChain(final NiFiUser user) {
+        // calculate the dn chain
+        final List<String> proxyChain = new ArrayList<>();
+
+        // build the dn chain
+        NiFiUser chainedUser = user;
+        while (chainedUser != null) {
+            // add the entry for this user
+            if (chainedUser.isAnonymous()) {
+                // use an empty string to represent an anonymous user in the proxy entities chain
+                proxyChain.add(StringUtils.EMPTY);
+            } else {
+                proxyChain.add(chainedUser.getIdentity());
+            }
+
+            // go to the next user in the chain
+            chainedUser = chainedUser.getChain();
+        }
+
+        return proxyChain;
+    }
+}