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/06 18:54:06 UTC

[nifi-registry] branch main updated: NIFIREG-408 Initial commit adding TenantsClient and PoliciesClient

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 4dde23b  NIFIREG-408 Initial commit adding TenantsClient and PoliciesClient
4dde23b is described below

commit 4dde23b664fe2595723a90c32632cfe8e285dda8
Author: Bryan Bende <bb...@apache.org>
AuthorDate: Mon Jul 27 13:42:22 2020 -0400

    NIFIREG-408 Initial commit adding TenantsClient and PoliciesClient
    
    NIFIREG-408 Separate tenants IT into separate tests for users and groups
    
    This closes #292.
    
    Signed-off-by: Kevin Doran <kd...@apache.org>
---
 .../nifi/registry/client/NiFiRegistryClient.java   |  54 +++++++
 .../nifi/registry/client/PoliciesClient.java       |  62 ++++++++
 .../apache/nifi/registry/client/TenantsClient.java | 166 +++++++++++++++++++++
 .../client/impl/AbstractCRUDJerseyClient.java      | 111 ++++++++++++++
 .../client/impl/JerseyNiFiRegistryClient.java      |  24 +++
 .../registry/client/impl/JerseyPoliciesClient.java |  65 ++++++++
 .../registry/client/impl/JerseyTenantsClient.java  | 113 ++++++++++++++
 .../web/api/SecureNiFiRegistryClientIT.java        | 149 ++++++++++++++----
 8 files changed, 712 insertions(+), 32 deletions(-)

diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java
index e141136..a7497a1 100644
--- a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/NiFiRegistryClient.java
@@ -33,6 +33,8 @@ public interface NiFiRegistryClient extends Closeable {
      */
     BucketClient getBucketClient(String ... proxiedEntity);
 
+    //-------------------------------------------------------------------------------------------
+
     /**
      * @return the client for interacting with flows
      */
@@ -43,6 +45,8 @@ public interface NiFiRegistryClient extends Closeable {
      */
     FlowClient getFlowClient(String ... proxiedEntity);
 
+    //-------------------------------------------------------------------------------------------
+
     /**
      * @return the client for interacting with flows/snapshots
      */
@@ -53,6 +57,8 @@ public interface NiFiRegistryClient extends Closeable {
      */
     FlowSnapshotClient getFlowSnapshotClient(String ... proxiedEntity);
 
+    //-------------------------------------------------------------------------------------------
+
     /**
      * @return the client for interacting with bucket items
      */
@@ -63,6 +69,8 @@ public interface NiFiRegistryClient extends Closeable {
      */
     ItemsClient getItemsClient(String ... proxiedEntity);
 
+    //-------------------------------------------------------------------------------------------
+
     /**
      * @return the client for obtaining information about the current user
      */
@@ -73,6 +81,8 @@ public interface NiFiRegistryClient extends Closeable {
      */
     UserClient getUserClient(String ... proxiedEntity);
 
+    //-------------------------------------------------------------------------------------------
+
     /**
      * @return the client for interacting with extension bundles
      */
@@ -83,6 +93,8 @@ public interface NiFiRegistryClient extends Closeable {
      */
     BundleClient getBundleClient(String ... proxiedEntity);
 
+    //-------------------------------------------------------------------------------------------
+
     /**
      * @return the client for interacting with extension bundle versions
      */
@@ -93,6 +105,8 @@ public interface NiFiRegistryClient extends Closeable {
      */
     BundleVersionClient getBundleVersionClient(String ... proxiedEntity);
 
+    //-------------------------------------------------------------------------------------------
+
     /**
      * @return the client for interacting with the extension repository
      */
@@ -103,6 +117,8 @@ public interface NiFiRegistryClient extends Closeable {
      */
     ExtensionRepoClient getExtensionRepoClient(String ... proxiedEntity);
 
+    //-------------------------------------------------------------------------------------------
+
     /**
      * @return the client for interacting with extensions
      */
@@ -113,6 +129,44 @@ public interface NiFiRegistryClient extends Closeable {
      */
     ExtensionClient getExtensionClient(String ... proxiedEntity);
 
+    //-------------------------------------------------------------------------------------------
+
+    /**
+     * Returns client for interacting with tenants.
+     *
+     * @return the client for interacting with tenants
+     */
+    TenantsClient getTenantsClient();
+
+    /**
+     * Returns client for interacting with tenants.
+     *
+     * @param proxiedEntity The given proxied entities.
+     *
+     * @return the client for interacting with tenants on behalf of the given proxied entities.
+     */
+    TenantsClient getTenantsClient(String ... proxiedEntity);
+
+    //-------------------------------------------------------------------------------------------
+
+    /**
+     * Returns client for interacting with access policies.
+     *
+     * @return the client for interacting with access policies
+     */
+    PoliciesClient getPoliciesClient();
+
+    /**
+     * Returns client for interacting with access policies.
+     *
+     * @param proxiedEntity The given proxied entities.
+     *
+     * @return the client for interacting with access policies on behalf of the given proxied entities.
+     */
+    PoliciesClient getPoliciesClient(String ... proxiedEntity);
+
+    //-------------------------------------------------------------------------------------------
+
     /**
      * The builder interface that implementations should provide for obtaining the client.
      */
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/PoliciesClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/PoliciesClient.java
new file mode 100644
index 0000000..00a46e2
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/PoliciesClient.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.client;
+
+import org.apache.nifi.registry.authorization.AccessPolicy;
+
+import java.io.IOException;
+
+public interface PoliciesClient {
+
+    /**
+     * Returns a given access policy.
+     *
+     * @param resource The action allowed by the access policy.
+     * @param action The resource managed by the access policy.
+     *
+     * @return The access policy.
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    AccessPolicy getAccessPolicy(String action, String resource) throws NiFiRegistryException, IOException;
+
+    /**
+     * Creates a new access policy.
+     *
+     * @param policy The access policy to be created. Note: identifier will be ignored and assigned by NiFi Registry.
+     *
+     * @return The created access with an assigned identifier.
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    AccessPolicy createAccessPolicy(AccessPolicy policy) throws NiFiRegistryException, IOException;
+
+    /**
+     * Updates an existing access policy.
+     *
+     * @param policy The access policy with new attributes.
+     *
+     * @return The updated access policy.
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    AccessPolicy updateAccessPolicy(AccessPolicy policy) throws NiFiRegistryException, IOException;
+
+}
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/TenantsClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/TenantsClient.java
new file mode 100644
index 0000000..adb117c
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/TenantsClient.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.client;
+
+import org.apache.nifi.registry.authorization.User;
+import org.apache.nifi.registry.authorization.UserGroup;
+import org.apache.nifi.registry.revision.entity.RevisionInfo;
+
+import java.io.IOException;
+import java.util.List;
+
+public interface TenantsClient {
+
+    /**
+     * Returns all users.
+     *
+     * @return The list of users.
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    List<User> getUsers() throws NiFiRegistryException, IOException;
+
+    /**
+     * Returns a user with a given identifier.
+     *
+     * @param id Identifier of the user.
+     *
+     * @return The user.
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    User getUser(String id) throws NiFiRegistryException, IOException;
+
+    /**
+     * Creates a new user in NiFi Registry.
+     *
+     * @param user The new user. Note: identifier will be ignored and assigned be NiFi Registry.
+     *
+     * @return The created user with an assigned identifier.
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    User createUser(User user) throws NiFiRegistryException, IOException;
+
+    /**
+     * Updates an existing user.
+     *
+     * @param user The user with the new attributes.
+     *
+     * @return The updated user.
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    User updateUser(User user) throws NiFiRegistryException, IOException;
+
+    /**
+     * Deletes an existing user.
+     *
+     * @param id identifier of the user
+     * @return the deleted user
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    User deleteUser(String id) throws NiFiRegistryException, IOException;
+
+    /**
+     * Deletes an existing user.
+     *
+     * @param id identifier of the user
+     * @param revisionInfo the revision info for the user to delete
+     * @return the deleted user
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    User deleteUser(String id, RevisionInfo revisionInfo) throws NiFiRegistryException, IOException;
+
+    /**
+     * Returns all user groups.
+     *
+     * @return The list of user groups.
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    List<UserGroup> getUserGroups() throws NiFiRegistryException, IOException;
+
+    /**
+     * Returns a user group with a given identifier.
+     *
+     * @param id Identifier of the user group.
+     *
+     * @return The user group.
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    UserGroup getUserGroup(String id) throws NiFiRegistryException, IOException;
+
+    /**
+     * Creates a new user group.
+     *
+     * @param group The user group to be created. Note: identifier will be ignored and assigned by NiFi Registry.
+     *
+     * @return The created user group with an assigned identifier.
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    UserGroup createUserGroup(UserGroup group) throws NiFiRegistryException, IOException;
+
+    /**
+     * Updates an existing user group.
+     *
+     * @param group The user group with new attributes.
+     *
+     * @return The user group after store.
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    UserGroup updateUserGroup(UserGroup group) throws NiFiRegistryException, IOException;
+
+    /**
+     * Deletes an existing group.
+     *
+     * @param id identifier of the group
+     * @return the deleted group
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    UserGroup deleteUserGroup(String id) throws NiFiRegistryException, IOException;
+
+    /**
+     * Deletes an existing group.
+     *
+     * @param id identifier of the group
+     * @param revisionInfo the revision info for the group to delete
+     * @return the deleted group
+     *
+     * @throws NiFiRegistryException Thrown in case of unsuccessful execution.
+     * @throws IOException Thrown when there is an issue while communicating with NiFi Registry.
+     */
+    UserGroup deleteUserGroup(String id, RevisionInfo revisionInfo) throws NiFiRegistryException, IOException;
+
+}
\ No newline at end of file
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/AbstractCRUDJerseyClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/AbstractCRUDJerseyClient.java
new file mode 100644
index 0000000..a6f9ac0
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/AbstractCRUDJerseyClient.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.registry.client.impl;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.revision.entity.RevisionInfo;
+
+import javax.ws.rs.client.Entity;
+import javax.ws.rs.client.WebTarget;
+import javax.ws.rs.core.MediaType;
+import java.io.IOException;
+import java.util.Map;
+
+public class AbstractCRUDJerseyClient extends AbstractJerseyClient {
+
+    protected final WebTarget baseTarget;
+
+    public AbstractCRUDJerseyClient(final WebTarget baseTarget, final Map<String, String> headers) {
+        super(headers);
+        this.baseTarget = baseTarget;
+    }
+
+    protected <T> T get(
+            String id,
+            Class<T> entityType,
+            String entityTypeName,
+            String entityPath
+    ) throws NiFiRegistryException, IOException {
+        if (StringUtils.isBlank(id)) {
+            throw new IllegalArgumentException(entityTypeName + " id cannot be blank");
+        }
+
+        return executeAction("Error retrieving " + entityTypeName.toLowerCase(), () -> {
+            final WebTarget target = baseTarget.path(entityPath).path(id);
+            return getRequestBuilder(target).get(entityType);
+        });
+    }
+
+    protected <T> T create(
+            T entity,
+            Class<T> entityType,
+            String entityTypeName,
+            String entityPath
+    ) throws NiFiRegistryException, IOException {
+        if (entity == null) {
+            throw new IllegalArgumentException(entityTypeName + " cannot be null");
+        }
+
+        return executeAction("Error creating " + entityTypeName.toLowerCase(), () -> {
+            final WebTarget target = baseTarget.path(entityPath);
+
+            return getRequestBuilder(target).post(
+                    Entity.entity(entity, MediaType.APPLICATION_JSON_TYPE), entityType
+            );
+        });
+    }
+
+    protected <T> T update(
+            T entity,
+            String id,
+            Class<T> entityType,
+            String entityTypeName,
+            String entityPath
+    ) throws NiFiRegistryException, IOException {
+        if (entity == null) {
+            throw new IllegalArgumentException(entityTypeName + " cannot be null");
+        }
+
+        return executeAction("Error updating " + entityTypeName.toLowerCase(), () -> {
+            final WebTarget target = baseTarget.path(entityPath).path(id);
+
+            return getRequestBuilder(target).put(
+                    Entity.entity(entity, MediaType.APPLICATION_JSON_TYPE), entityType
+            );
+        });
+    }
+
+    protected <T> T delete(
+            String id,
+            RevisionInfo revisionInfo,
+            Class<T> entityType,
+            String entityTypeName,
+            String entityPath
+    ) throws NiFiRegistryException, IOException {
+        if (StringUtils.isBlank(id)) {
+            throw new IllegalArgumentException(entityTypeName + " id cannot be blank");
+        }
+
+        return executeAction("Error deleting " + entityTypeName.toLowerCase(), () -> {
+            WebTarget target = baseTarget.path(entityPath).path(id);
+            target = addRevisionQueryParams(target, revisionInfo);
+
+            return getRequestBuilder(target).delete(entityType);
+        });
+    }
+}
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
index 739d225..f876277 100644
--- a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyNiFiRegistryClient.java
@@ -33,6 +33,8 @@ import org.apache.nifi.registry.client.FlowSnapshotClient;
 import org.apache.nifi.registry.client.ItemsClient;
 import org.apache.nifi.registry.client.NiFiRegistryClient;
 import org.apache.nifi.registry.client.NiFiRegistryClientConfig;
+import org.apache.nifi.registry.client.PoliciesClient;
+import org.apache.nifi.registry.client.TenantsClient;
 import org.apache.nifi.registry.client.UserClient;
 import org.apache.nifi.registry.security.util.ProxiedEntitiesUtils;
 import org.glassfish.jersey.client.ClientConfig;
@@ -227,6 +229,28 @@ public class JerseyNiFiRegistryClient implements NiFiRegistryClient {
         return new JerseyExtensionRepoClient(baseTarget, headers);
     }
 
+    @Override
+    public TenantsClient getTenantsClient() {
+        return new JerseyTenantsClient(baseTarget);
+    }
+
+    @Override
+    public TenantsClient getTenantsClient(String... proxiedEntity) {
+        final Map<String,String> headers = getHeaders(proxiedEntity);
+        return new JerseyTenantsClient(baseTarget, headers);
+    }
+
+    @Override
+    public PoliciesClient getPoliciesClient() {
+        return new JerseyPoliciesClient(baseTarget);
+    }
+
+    @Override
+    public PoliciesClient getPoliciesClient(String... proxiedEntity) {
+        final Map<String,String> headers = getHeaders(proxiedEntity);
+        return new JerseyPoliciesClient(baseTarget, headers);
+    }
+
     private Map<String,String> getHeaders(String[] proxiedEntities) {
         final String proxiedEntitiesValue = getProxiedEntitesValue(proxiedEntities);
 
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyPoliciesClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyPoliciesClient.java
new file mode 100644
index 0000000..15577f6
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyPoliciesClient.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.client.impl;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.authorization.AccessPolicy;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.client.PoliciesClient;
+
+import javax.ws.rs.client.WebTarget;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+public class JerseyPoliciesClient extends AbstractCRUDJerseyClient implements PoliciesClient {
+
+    public static final String ACCESS_POLICY = "Access policy";
+    public static final String POLICIES_PATH = "policies";
+
+    public JerseyPoliciesClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyPoliciesClient(final WebTarget baseTarget, final Map<String, String> headers) {
+        super(baseTarget, headers);
+    }
+
+    @Override
+    public AccessPolicy getAccessPolicy(String action, String resource) throws NiFiRegistryException, IOException {
+        if (StringUtils.isBlank(resource) || StringUtils.isBlank(action)) {
+            throw new IllegalArgumentException("Resource and action cannot be null");
+        }
+
+        return executeAction("Error retrieving access policy", () -> {
+            final WebTarget target = baseTarget.path(POLICIES_PATH).path(action).path(resource);
+            return getRequestBuilder(target).get(AccessPolicy.class);
+        });
+    }
+
+    @Override
+    public AccessPolicy createAccessPolicy(final AccessPolicy policy) throws NiFiRegistryException, IOException {
+        return create(policy, AccessPolicy.class, ACCESS_POLICY, POLICIES_PATH);
+    }
+
+    @Override
+    public AccessPolicy updateAccessPolicy(final AccessPolicy policy) throws NiFiRegistryException, IOException {
+        return update(policy, policy.getIdentifier(), AccessPolicy.class, ACCESS_POLICY, POLICIES_PATH);
+    }
+
+}
+
diff --git a/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyTenantsClient.java b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyTenantsClient.java
new file mode 100644
index 0000000..78867c7
--- /dev/null
+++ b/nifi-registry-core/nifi-registry-client/src/main/java/org/apache/nifi/registry/client/impl/JerseyTenantsClient.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.client.impl;
+
+import org.apache.nifi.registry.authorization.User;
+import org.apache.nifi.registry.authorization.UserGroup;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.client.TenantsClient;
+import org.apache.nifi.registry.revision.entity.RevisionInfo;
+
+import javax.ws.rs.client.WebTarget;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class JerseyTenantsClient extends AbstractCRUDJerseyClient implements TenantsClient {
+    public static final String USER = "User";
+    public static final String USERS_PATH = "users";
+
+    public static final String USER_GROUP = "User group";
+    public static final String USER_GROUPS_PATH = "user-groups";
+
+    public JerseyTenantsClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    public JerseyTenantsClient(final WebTarget baseTarget, final Map<String, String> headers) {
+        super(baseTarget.path("/tenants"), headers);
+    }
+
+    @Override
+    public List<User> getUsers() throws NiFiRegistryException, IOException {
+        return executeAction("Error retrieving users", () -> {
+            final WebTarget target = baseTarget.path(USERS_PATH);
+            return Arrays.asList(getRequestBuilder(target).get(User[].class));
+        });
+    }
+
+    @Override
+    public User getUser(final String id) throws NiFiRegistryException, IOException {
+        return get(id, User.class, USER, USERS_PATH);
+    }
+
+    @Override
+    public User createUser(final User user) throws NiFiRegistryException, IOException {
+        return create(user, User.class, USER, USERS_PATH);
+    }
+
+    @Override
+    public User updateUser(final User user) throws NiFiRegistryException, IOException {
+        return update(user, user.getIdentifier(), User.class, USER, USERS_PATH);
+    }
+
+    @Override
+    public User deleteUser(final String id) throws NiFiRegistryException, IOException {
+        return delete(id, null, User.class, USER, USERS_PATH);
+    }
+
+    @Override
+    public User deleteUser(final String id, final RevisionInfo revisionInfo) throws NiFiRegistryException, IOException {
+        return delete(id, revisionInfo, User.class, USER, USERS_PATH);
+    }
+
+    @Override
+    public List<UserGroup> getUserGroups() throws NiFiRegistryException, IOException {
+        return executeAction("Error retrieving users", () -> {
+            final WebTarget target = baseTarget.path(USER_GROUPS_PATH);
+            return Arrays.asList(getRequestBuilder(target).get(UserGroup[].class));
+        });
+    }
+
+    @Override
+    public UserGroup getUserGroup(final String id) throws NiFiRegistryException, IOException {
+        return get(id, UserGroup.class, USER_GROUP, USER_GROUPS_PATH);
+    }
+
+    @Override
+    public UserGroup createUserGroup(final UserGroup group) throws NiFiRegistryException, IOException {
+        return create(group, UserGroup.class, USER_GROUP, USER_GROUPS_PATH);
+    }
+
+    @Override
+    public UserGroup updateUserGroup(final UserGroup group) throws NiFiRegistryException, IOException {
+        return update(group, group.getIdentifier(), UserGroup.class, USER_GROUP, USER_GROUPS_PATH);
+    }
+
+    @Override
+    public UserGroup deleteUserGroup(final String id) throws NiFiRegistryException, IOException {
+        return delete(id, null, UserGroup.class, USER_GROUP, USER_GROUPS_PATH);
+    }
+
+    @Override
+    public UserGroup deleteUserGroup(final String id, final RevisionInfo revisionInfo) throws NiFiRegistryException, IOException {
+        return delete(id, revisionInfo, UserGroup.class, USER_GROUP, USER_GROUPS_PATH);
+    }
+
+}
diff --git a/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureNiFiRegistryClientIT.java b/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureNiFiRegistryClientIT.java
index dfffd7d..e49edf2 100644
--- a/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureNiFiRegistryClientIT.java
+++ b/nifi-registry-core/nifi-registry-web-api/src/test/java/org/apache/nifi/registry/web/api/SecureNiFiRegistryClientIT.java
@@ -17,7 +17,10 @@
 package org.apache.nifi.registry.web.api;
 
 import org.apache.nifi.registry.NiFiRegistryTestApiApplication;
+import org.apache.nifi.registry.authorization.CurrentUser;
 import org.apache.nifi.registry.authorization.Permissions;
+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.FlowClient;
@@ -25,13 +28,13 @@ import org.apache.nifi.registry.client.FlowSnapshotClient;
 import org.apache.nifi.registry.client.NiFiRegistryClient;
 import org.apache.nifi.registry.client.NiFiRegistryClientConfig;
 import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.client.TenantsClient;
 import org.apache.nifi.registry.client.UserClient;
 import org.apache.nifi.registry.client.impl.JerseyNiFiRegistryClient;
 import org.apache.nifi.registry.flow.VersionedFlow;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshot;
 import org.apache.nifi.registry.flow.VersionedFlowSnapshotMetadata;
 import org.apache.nifi.registry.flow.VersionedProcessGroup;
-import org.apache.nifi.registry.authorization.CurrentUser;
 import org.apache.nifi.registry.revision.entity.RevisionInfo;
 import org.junit.After;
 import org.junit.Assert;
@@ -49,6 +52,12 @@ import javax.ws.rs.ForbiddenException;
 import java.io.IOException;
 import java.util.List;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 @RunWith(SpringRunner.class)
 @SpringBootTest(
         classes = NiFiRegistryTestApiApplication.class,
@@ -60,6 +69,9 @@ public class SecureNiFiRegistryClientIT extends IntegrationTestBase {
 
     static final Logger LOGGER = LoggerFactory.getLogger(SecureNiFiRegistryClientIT.class);
 
+    static final String INITIAL_ADMIN_IDENTITY = "CN=user1, OU=nifi";
+    static final String NO_ACCESS_IDENTITY = "CN=no-access, OU=nifi";
+
     private NiFiRegistryClient client;
 
     @Before
@@ -90,15 +102,15 @@ public class SecureNiFiRegistryClientIT extends IntegrationTestBase {
     public void testGetAccessStatus() throws IOException, NiFiRegistryException {
         final UserClient userClient = client.getUserClient();
         final CurrentUser currentUser = userClient.getAccessStatus();
-        Assert.assertEquals("CN=user1, OU=nifi", currentUser.getIdentity());
-        Assert.assertFalse(currentUser.isAnonymous());
-        Assert.assertNotNull(currentUser.getResourcePermissions());
+        assertEquals(INITIAL_ADMIN_IDENTITY, currentUser.getIdentity());
+        assertFalse(currentUser.isAnonymous());
+        assertNotNull(currentUser.getResourcePermissions());
         Permissions fullAccess = new Permissions().withCanRead(true).withCanWrite(true).withCanDelete(true);
-        Assert.assertEquals(fullAccess, currentUser.getResourcePermissions().getAnyTopLevelResource());
-        Assert.assertEquals(fullAccess, currentUser.getResourcePermissions().getBuckets());
-        Assert.assertEquals(fullAccess, currentUser.getResourcePermissions().getTenants());
-        Assert.assertEquals(fullAccess, currentUser.getResourcePermissions().getPolicies());
-        Assert.assertEquals(new Permissions().withCanWrite(true).withCanRead(true).withCanDelete(true), currentUser.getResourcePermissions().getProxy());
+        assertEquals(fullAccess, currentUser.getResourcePermissions().getAnyTopLevelResource());
+        assertEquals(fullAccess, currentUser.getResourcePermissions().getBuckets());
+        assertEquals(fullAccess, currentUser.getResourcePermissions().getTenants());
+        assertEquals(fullAccess, currentUser.getResourcePermissions().getPolicies());
+        assertEquals(new Permissions().withCanWrite(true).withCanRead(true).withCanDelete(true), currentUser.getResourcePermissions().getProxy());
     }
 
     @Test
@@ -110,13 +122,13 @@ public class SecureNiFiRegistryClientIT extends IntegrationTestBase {
 
         final BucketClient bucketClient = client.getBucketClient();
         final Bucket createdBucket = bucketClient.create(bucket);
-        Assert.assertNotNull(createdBucket);
-        Assert.assertNotNull(createdBucket.getIdentifier());
-        Assert.assertNotNull(createdBucket.getRevision());
+        assertNotNull(createdBucket);
+        assertNotNull(createdBucket.getIdentifier());
+        assertNotNull(createdBucket.getRevision());
 
         final List<Bucket> buckets = bucketClient.getAll();
         Assert.assertEquals(4, buckets.size());
-        buckets.forEach(b -> Assert.assertNotNull(b.getRevision()));
+        buckets.forEach(b -> assertNotNull(b.getRevision()));
 
         final VersionedFlow flow = new VersionedFlow();
         flow.setBucketIdentifier(createdBucket.getIdentifier());
@@ -125,9 +137,9 @@ public class SecureNiFiRegistryClientIT extends IntegrationTestBase {
 
         final FlowClient flowClient = client.getFlowClient();
         final VersionedFlow createdFlow = flowClient.create(flow);
-        Assert.assertNotNull(createdFlow);
-        Assert.assertNotNull(createdFlow.getIdentifier());
-        Assert.assertNotNull(createdFlow.getRevision());
+        assertNotNull(createdFlow);
+        assertNotNull(createdFlow.getIdentifier());
+        assertNotNull(createdFlow.getRevision());
 
         final VersionedFlowSnapshotMetadata snapshotMetadata = new VersionedFlowSnapshotMetadata();
         snapshotMetadata.setBucketIdentifier(createdFlow.getBucketIdentifier());
@@ -145,8 +157,8 @@ public class SecureNiFiRegistryClientIT extends IntegrationTestBase {
 
         final FlowSnapshotClient snapshotClient = client.getFlowSnapshotClient();
         final VersionedFlowSnapshot createdSnapshot = snapshotClient.create(snapshot);
-        Assert.assertNotNull(createdSnapshot);
-        Assert.assertEquals("CN=user1, OU=nifi", createdSnapshot.getSnapshotMetadata().getAuthor());
+        assertNotNull(createdSnapshot);
+        assertEquals(INITIAL_ADMIN_IDENTITY, createdSnapshot.getSnapshotMetadata().getAuthor());
     }
 
     @Test
@@ -154,8 +166,8 @@ public class SecureNiFiRegistryClientIT extends IntegrationTestBase {
         final String proxiedEntity = "user2";
         final UserClient userClient = client.getUserClient(proxiedEntity);
         final CurrentUser status = userClient.getAccessStatus();
-        Assert.assertEquals("user2", status.getIdentity());
-        Assert.assertFalse(status.isAnonymous());
+        assertEquals("user2", status.getIdentity());
+        assertFalse(status.isAnonymous());
     }
 
     @Test
@@ -170,7 +182,7 @@ public class SecureNiFiRegistryClientIT extends IntegrationTestBase {
 
         try {
             bucketClient.create(bucket);
-            Assert.fail("Shouldn't have been able to create a bucket");
+            fail("Shouldn't have been able to create a bucket");
         } catch (Exception e) {
 
         }
@@ -179,46 +191,119 @@ public class SecureNiFiRegistryClientIT extends IntegrationTestBase {
     @Test
     public void testDirectFlowAccess() throws IOException {
         // this user shouldn't have access to anything
-        final String proxiedEntity = "CN=no-access, OU=nifi";
+        final String proxiedEntity = NO_ACCESS_IDENTITY;
 
         final FlowClient proxiedFlowClient = client.getFlowClient(proxiedEntity);
         final FlowSnapshotClient proxiedFlowSnapshotClient = client.getFlowSnapshotClient(proxiedEntity);
 
         try {
             proxiedFlowClient.get("1");
-            Assert.fail("Shouldn't have been able to retrieve flow");
+            fail("Shouldn't have been able to retrieve flow");
         } catch (NiFiRegistryException e) {
-            Assert.assertTrue(e.getCause()  instanceof ForbiddenException);
+            assertTrue(e.getCause()  instanceof ForbiddenException);
         }
 
         try {
             proxiedFlowSnapshotClient.getLatest("1");
-            Assert.fail("Shouldn't have been able to retrieve flow");
+            fail("Shouldn't have been able to retrieve flow");
         } catch (NiFiRegistryException e) {
-            Assert.assertTrue(e.getCause()  instanceof ForbiddenException);
+            assertTrue(e.getCause()  instanceof ForbiddenException);
         }
 
         try {
             proxiedFlowSnapshotClient.getLatestMetadata("1");
-            Assert.fail("Shouldn't have been able to retrieve flow");
+            fail("Shouldn't have been able to retrieve flow");
         } catch (NiFiRegistryException e) {
-            Assert.assertTrue(e.getCause()  instanceof ForbiddenException);
+            assertTrue(e.getCause()  instanceof ForbiddenException);
         }
 
         try {
             proxiedFlowSnapshotClient.get("1", 1);
-            Assert.fail("Shouldn't have been able to retrieve flow");
+            fail("Shouldn't have been able to retrieve flow");
         } catch (NiFiRegistryException e) {
-            Assert.assertTrue(e.getCause()  instanceof ForbiddenException);
+            assertTrue(e.getCause()  instanceof ForbiddenException);
         }
 
         try {
             proxiedFlowSnapshotClient.getSnapshotMetadata("1");
-            Assert.fail("Shouldn't have been able to retrieve flow");
+            fail("Shouldn't have been able to retrieve flow");
         } catch (NiFiRegistryException e) {
-            Assert.assertTrue(e.getCause()  instanceof ForbiddenException);
+            assertTrue(e.getCause()  instanceof ForbiddenException);
         }
 
     }
 
+    @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());
+
+    }
 }