You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by kd...@apache.org on 2020/08/10 16:59:45 UTC

[nifi-registry] branch main updated: NIFIREG-410 Add integration tests that cover the new Database UserGroupProvider and AccessPolicyProvider

This is an automated email from the ASF dual-hosted git repository.

kdoran pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi-registry.git


The following commit(s) were added to refs/heads/main by this push:
     new 85a3243  NIFIREG-410 Add integration tests that cover the new Database UserGroupProvider and AccessPolicyProvider
85a3243 is described below

commit 85a3243ca0cfa3244d8852a636fd667715e7412d
Author: Bryan Bende <bb...@apache.org>
AuthorDate: Mon Aug 10 11:19:10 2020 -0400

    NIFIREG-410 Add integration tests that cover the new Database UserGroupProvider and AccessPolicyProvider
    
    This closes #294.
    
    Signed-off-by: Kevin Doran <kd...@apache.org>
---
 .../nifi/registry/web/api/SecureDatabaseIT.java    | 238 +++++++++++++++++++++
 .../application-ITSecureDatabase.properties        |  36 ++++
 .../resources/conf/secure-database/authorizers.xml |  80 +++++++
 .../nifi-registry-client.properties                |  25 +++
 .../conf/secure-database/nifi-registry.properties  |  33 +++
 .../src/test/resources/db/clearDB.sql              |   2 +
 6 files changed, 414 insertions(+)

diff --git a/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureDatabaseIT.java b/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureDatabaseIT.java
new file mode 100644
index 0000000..3b7ce60
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureDatabaseIT.java
@@ -0,0 +1,238 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.nifi.registry.NiFiRegistryTestApiApplication;
+import org.apache.nifi.registry.authorization.AccessPolicy;
+import org.apache.nifi.registry.authorization.User;
+import org.apache.nifi.registry.authorization.UserGroup;
+import org.apache.nifi.registry.bucket.Bucket;
+import org.apache.nifi.registry.client.BucketClient;
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.registry.client.NiFiRegistryClientConfig;
+import org.apache.nifi.registry.client.PoliciesClient;
+import org.apache.nifi.registry.client.TenantsClient;
+import org.apache.nifi.registry.client.impl.JerseyNiFiRegistryClient;
+import org.apache.nifi.registry.revision.entity.RevisionInfo;
+import org.apache.nifi.registry.security.authorization.RequestAction;
+import org.apache.nifi.registry.security.authorization.resource.ResourceFactory;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.context.annotation.Import;
+import org.springframework.test.context.jdbc.Sql;
+import org.springframework.test.context.junit4.SpringRunner;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+@RunWith(SpringRunner.class)
+@SpringBootTest(
+        classes = NiFiRegistryTestApiApplication.class,
+        webEnvironment = SpringBootTest.WebEnvironment.RANDOM_PORT,
+        properties = "spring.profiles.include=ITSecureDatabase")
+@Import(SecureITClientConfiguration.class)
+@Sql(executionPhase = Sql.ExecutionPhase.BEFORE_TEST_METHOD, scripts = {"classpath:db/clearDB.sql"})
+public class SecureDatabaseIT extends IntegrationTestBase {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(SecureDatabaseIT.class);
+
+    private static final String INITIAL_ADMIN_IDENTITY = "CN=user1, OU=nifi";
+    private static final String OTHER_USER_IDENTITY = "CN=user2, OU=nifi";
+
+    private NiFiRegistryClient client;
+
+    @Before
+    public void setup() {
+        final String baseUrl = createBaseURL();
+        LOGGER.info("Using base url = " + baseUrl);
+
+        final NiFiRegistryClientConfig clientConfig = createClientConfig(baseUrl);
+        Assert.assertNotNull(clientConfig);
+
+        final NiFiRegistryClient client = new JerseyNiFiRegistryClient.Builder()
+                .config(clientConfig)
+                .build();
+        Assert.assertNotNull(client);
+        this.client = client;
+    }
+
+    @After
+    public void teardown() {
+        try {
+            client.close();
+        } catch (Exception e) {
+
+        }
+    }
+
+    @Test
+    public void testTenantsClientUsers() throws Exception {
+        final TenantsClient tenantsClient = client.getTenantsClient();
+
+        // get all users
+        final List<User> users = tenantsClient.getUsers();
+        assertEquals(2, users.size());
+
+        final User initialAdminUser = users.stream()
+                .filter(u -> u.getIdentity().equals(INITIAL_ADMIN_IDENTITY))
+                .findFirst()
+                .orElse(null);
+        assertNotNull(initialAdminUser);
+
+        // get user by id
+        final User retrievedInitialAdminUser = tenantsClient.getUser(initialAdminUser.getIdentifier());
+        assertNotNull(retrievedInitialAdminUser);
+        assertEquals(initialAdminUser.getIdentity(), retrievedInitialAdminUser.getIdentity());
+
+        // add user
+        final User userToAdd = new User();
+        userToAdd.setIdentity("some-new-user");
+        userToAdd.setRevision(new RevisionInfo(null, 0L));
+
+        final User createdUser = tenantsClient.createUser(userToAdd);
+        assertNotNull(createdUser);
+        assertEquals(3, tenantsClient.getUsers().size());
+
+        // update user
+        createdUser.setIdentity(createdUser.getIdentity() + "-updated");
+        final User updatedUser = tenantsClient.updateUser(createdUser);
+        assertNotNull(updatedUser);
+        assertEquals(createdUser.getIdentity(), updatedUser.getIdentity());
+
+        // delete user
+        final User deletedUser = tenantsClient.deleteUser(updatedUser.getIdentifier(), updatedUser.getRevision());
+        assertNotNull(deletedUser);
+        assertEquals(updatedUser.getIdentifier(), deletedUser.getIdentifier());
+    }
+
+    @Test
+    public void testTenantsClientGroups() throws Exception {
+        final TenantsClient tenantsClient = client.getTenantsClient();
+
+        // get all groups
+        final List<UserGroup> groups = tenantsClient.getUserGroups();
+        assertEquals(0, groups.size());
+
+        // create group
+        final UserGroup userGroup = new UserGroup();
+        userGroup.setIdentity("some-new group");
+        userGroup.setRevision(new RevisionInfo(null, 0L));
+
+        final UserGroup createdGroup = tenantsClient.createUserGroup(userGroup);
+        assertNotNull(createdGroup);
+        assertEquals(userGroup.getIdentity(), createdGroup.getIdentity());
+
+        // get group by id
+        final UserGroup retrievedGroup = tenantsClient.getUserGroup(createdGroup.getIdentifier());
+        assertNotNull(retrievedGroup);
+        assertEquals(createdGroup.getIdentifier(), retrievedGroup.getIdentifier());
+
+        // update group
+        retrievedGroup.setIdentity(retrievedGroup.getIdentity() + "-updated");
+        final UserGroup updatedGroup = tenantsClient.updateUserGroup(retrievedGroup);
+        assertEquals(retrievedGroup.getIdentity(), updatedGroup.getIdentity());
+
+        // delete group
+        final UserGroup deletedGroup = tenantsClient.deleteUserGroup(updatedGroup.getIdentifier(), updatedGroup.getRevision());
+        assertNotNull(deletedGroup);
+        assertEquals(retrievedGroup.getIdentifier(), deletedGroup.getIdentifier());
+    }
+
+    @Test
+    public void testPoliciesClient() throws Exception {
+        // Create a bucket...
+        final Bucket bucket = new Bucket();
+        bucket.setName("Bucket 1 " + System.currentTimeMillis());
+        bucket.setDescription("This is bucket 1");
+        bucket.setRevision(new RevisionInfo(null, 0L));
+
+        final BucketClient bucketClient = client.getBucketClient();
+        final Bucket createdBucket = bucketClient.create(bucket);
+        assertNotNull(createdBucket);
+        assertNotNull(createdBucket.getIdentifier());
+        assertNotNull(createdBucket.getRevision());
+
+        // Get initial users...
+        final TenantsClient tenantsClient = client.getTenantsClient();
+
+        final List<User> users = tenantsClient.getUsers();
+        assertEquals(2, users.size());
+
+        final User initialAdminUser = users.stream()
+                .filter(u -> u.getIdentity().equals(INITIAL_ADMIN_IDENTITY))
+                .findFirst()
+                .orElse(null);
+        assertNotNull(initialAdminUser);
+
+        final User otherUser = users.stream()
+                .filter(u -> u.getIdentity().equals(OTHER_USER_IDENTITY))
+                .findFirst()
+                .orElse(null);
+        assertNotNull(otherUser);
+
+        // Create a policy on the bucket...
+        final PoliciesClient policiesClient = client.getPoliciesClient();
+
+        final AccessPolicy readBucketAccessPolicy = new AccessPolicy();
+        readBucketAccessPolicy.setResource(ResourceFactory.getBucketResource(
+                createdBucket.getIdentifier(), createdBucket.getName())
+                .getIdentifier());
+        readBucketAccessPolicy.setAction(RequestAction.READ.toString());
+        readBucketAccessPolicy.setUsers(Collections.singleton(initialAdminUser));
+        readBucketAccessPolicy.setRevision(new RevisionInfo(null, 0L));
+
+        final AccessPolicy createdAccessPolicy = policiesClient.createAccessPolicy(readBucketAccessPolicy);
+        assertNotNull(createdAccessPolicy);
+        assertEquals(readBucketAccessPolicy.getAction(), createdAccessPolicy.getAction());
+        assertEquals(readBucketAccessPolicy.getResource(), createdAccessPolicy.getResource());
+        assertEquals(1, createdAccessPolicy.getUsers().size());
+        assertEquals(INITIAL_ADMIN_IDENTITY, createdAccessPolicy.getUsers().iterator().next().getIdentity());
+        assertEquals(1, createdAccessPolicy.getRevision().getVersion().longValue());
+
+        // Retrieve the policy by action + resource
+        final AccessPolicy retrievedAccessPolicy = policiesClient.getAccessPolicy(
+                createdAccessPolicy.getAction(), createdAccessPolicy.getResource());
+        assertNotNull(retrievedAccessPolicy);
+        assertEquals(createdAccessPolicy.getAction(), retrievedAccessPolicy.getAction());
+        assertEquals(createdAccessPolicy.getResource(), retrievedAccessPolicy.getResource());
+        assertEquals(1, retrievedAccessPolicy.getUsers().size());
+        assertEquals(INITIAL_ADMIN_IDENTITY, retrievedAccessPolicy.getUsers().iterator().next().getIdentity());
+        assertEquals(1, retrievedAccessPolicy.getRevision().getVersion().longValue());
+
+        // Update the policy
+        retrievedAccessPolicy.setUsers(new HashSet<>(Arrays.asList(initialAdminUser, otherUser)));
+
+        final AccessPolicy updatedAccessPolicy = policiesClient.updateAccessPolicy(retrievedAccessPolicy);
+        assertNotNull(updatedAccessPolicy);
+        assertEquals(retrievedAccessPolicy.getAction(), updatedAccessPolicy.getAction());
+        assertEquals(retrievedAccessPolicy.getResource(), updatedAccessPolicy.getResource());
+        assertEquals(2, updatedAccessPolicy.getUsers().size());
+        assertEquals(2, updatedAccessPolicy.getRevision().getVersion().longValue());
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-web-api/src/test/resources/application-ITSecureDatabase.properties b/nifi-registry-core/nifi-registry-web-api/src/test/resources/application-ITSecureDatabase.properties
new file mode 100644
index 0000000..b5e851e
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-web-api/src/test/resources/application-ITSecureDatabase.properties
@@ -0,0 +1,36 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+
+# Properties for Spring Boot integration tests
+# Documentation for common Spring Boot application properties can be found at:
+# https://docs.spring.io/spring-boot/docs/current/reference/html/common-application-properties.html
+
+
+# Custom (non-standard to Spring Boot) properties
+nifi.registry.properties.file: src/test/resources/conf/secure-database/nifi-registry.properties
+nifi.registry.client.properties.file: src/test/resources/conf/secure-database/nifi-registry-client.properties
+
+
+# Embedded Server SSL Context Config
+server.ssl.client-auth: need
+server.ssl.key-store: ./target/test-classes/keys/registry-ks.jks
+server.ssl.key-store-password: password
+server.ssl.key-password: password
+server.ssl.protocol: TLS
+server.ssl.trust-store: ./target/test-classes/keys/ca-ts.jks
+server.ssl.trust-store-password: password
\ No newline at end of file
diff --git a/nifi-registry-core/nifi-registry-web-api/src/test/resources/conf/secure-database/authorizers.xml b/nifi-registry-core/nifi-registry-web-api/src/test/resources/conf/secure-database/authorizers.xml
new file mode 100644
index 0000000..22e9ca3
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-web-api/src/test/resources/conf/secure-database/authorizers.xml
@@ -0,0 +1,80 @@
+<?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-registry.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 DatabaseUserGroupProvider will provide support for managing users and groups in a relational database. The framework
+        will provide a database connection to this provider using the same database information from nifi-registry.properties.
+
+        - Initial User Identity [unique key] - Same as the Initial User Identity in the FileUserGroupProvider
+    -->
+    <userGroupProvider>
+        <identifier>database-user-group-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.database.DatabaseUserGroupProvider</class>
+        <property name="Initial User Identity 1">CN=user1, OU=nifi</property>
+        <property name="Initial User Identity 2">CN=user2, OU=nifi</property>
+    </userGroupProvider>
+
+    <!--
+        The DatabaseAccessPolicyProvider will provide support for managing access policies in a relational database. The
+        framework will provide a database connection to this provider using the same database information from nifi-registry.properties.
+
+        - User Group Provider - Same as User Group Provider in the FileAccessPolicyProvider
+
+        - Initial Admin Identity - Same as Initial Admin Identity in the FileAccessPolicyProvider
+
+        - NiFi Identity [unique key] - Same as NiFi Identity in the FileAccessPolicyProvider
+
+        - NiFi Group Name - Same as NiFi Group Name in the FileAccessPolicyProvider
+    -->
+    <accessPolicyProvider>
+        <identifier>database-access-policy-provider</identifier>
+        <class>org.apache.nifi.registry.security.authorization.database.DatabaseAccessPolicyProvider</class>
+        <property name="User Group Provider">database-user-group-provider</property>
+        <property name="Initial Admin Identity">CN=user1, OU=nifi</property>
+        <property name="NiFi Identity 1"></property>
+        <property name="NiFi Group Name"></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">database-access-policy-provider</property>
+    </authorizer>
+
+</authorizers>
\ No newline at end of file
diff --git a/nifi-registry-core/nifi-registry-web-api/src/test/resources/conf/secure-database/nifi-registry-client.properties b/nifi-registry-core/nifi-registry-web-api/src/test/resources/conf/secure-database/nifi-registry-client.properties
new file mode 100644
index 0000000..5a31413
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-web-api/src/test/resources/conf/secure-database/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/user1-ks.jks
+nifi.registry.security.keystoreType=JKS
+nifi.registry.security.keystorePasswd=password
+nifi.registry.security.keyPasswd=password
+nifi.registry.security.truststore=./target/test-classes/keys/ca-ts.jks
+nifi.registry.security.truststoreType=JKS
+nifi.registry.security.truststorePasswd=password
diff --git a/nifi-registry-core/nifi-registry-web-api/src/test/resources/conf/secure-database/nifi-registry.properties b/nifi-registry-core/nifi-registry-web-api/src/test/resources/conf/secure-database/nifi-registry.properties
new file mode 100644
index 0000000..bbc16f8
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-web-api/src/test/resources/conf/secure-database/nifi-registry.properties
@@ -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.
+#
+
+# 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-database/authorizers.xml
+nifi.registry.security.authorizer=managed-authorizer
+
+# providers properties #
+nifi.registry.providers.configuration.file=./target/test-classes/conf/providers-db-flow-storage.xml
+
+# enabled revision checking #
+nifi.registry.revisions.enabled=true
\ No newline at end of file
diff --git a/nifi-registry-core/nifi-registry-web-api/src/test/resources/db/clearDB.sql b/nifi-registry-core/nifi-registry-web-api/src/test/resources/db/clearDB.sql
index 7661df5..2031b29 100644
--- a/nifi-registry-core/nifi-registry-web-api/src/test/resources/db/clearDB.sql
+++ b/nifi-registry-core/nifi-registry-web-api/src/test/resources/db/clearDB.sql
@@ -17,3 +17,5 @@ DELETE FROM FLOW_SNAPSHOT;
 DELETE FROM FLOW;
 DELETE FROM BUCKET_ITEM;
 DELETE FROM BUCKET;
+
+DELETE FROM FLOW_PERSISTENCE_PROVIDER;
\ No newline at end of file