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 2020/06/12 16:44:31 UTC

[nifi] 01/02: NIFI-7442 Add CLI commands to the registry in order to support automatic registry setup

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

bbende pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit d99983bdc70c8a91ea9b389fc576e6b84c76a27e
Author: Bence Simon <si...@gmail.com>
AuthorDate: Fri Jun 5 17:47:09 2020 +0200

    NIFI-7442 Add CLI commands to the registry in order to support automatic registry setup
---
 nifi-docs/src/main/asciidoc/toolkit-guide.adoc     |   6 +
 .../impl/client/ExtendedNiFiRegistryClient.java    |  61 +++++
 .../client/JerseyExtendedNiFiRegistryClient.java   | 300 +++++++++++++++++++++
 .../cli/impl/client/NiFiRegistryClientFactory.java |  37 ++-
 .../cli/impl/client/registry/PoliciesClient.java   |  64 +++++
 .../cli/impl/client/registry/TenantsClient.java    | 122 +++++++++
 .../client/registry/impl/JerseyPoliciesClient.java |  82 ++++++
 .../client/registry/impl/JerseyTenantsClient.java  | 146 ++++++++++
 .../toolkit/cli/impl/command/CommandOption.java    |   2 +
 .../command/registry/NiFiRegistryCommandGroup.java |  12 +
 .../impl/command/registry/tenant/CreatePolicy.java |  97 +++++++
 .../impl/command/registry/tenant/CreateUser.java   |  69 +++++
 .../command/registry/tenant/CreateUserGroup.java   |  83 ++++++
 .../impl/command/registry/tenant/TenantHelper.java |  93 +++++++
 .../impl/command/registry/tenant/UpdatePolicy.java | 112 ++++++++
 .../impl/command/registry/tenant/UpdateUser.java   |  79 ++++++
 .../command/registry/tenant/UpdateUserGroup.java   |  91 +++++++
 17 files changed, 1449 insertions(+), 7 deletions(-)

diff --git a/nifi-docs/src/main/asciidoc/toolkit-guide.adoc b/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
index 76b7333..44df219 100644
--- a/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
+++ b/nifi-docs/src/main/asciidoc/toolkit-guide.adoc
@@ -106,6 +106,12 @@ The following are available commands:
  registry import-flow-version
  registry sync-flow-versions
  registry transfer-flow-version
+ registry create-user
+ registry update-user
+ registry create-user-group
+ registry update-user-group
+ registry create-policy
+ registry update-policy
  session keys
  session show
  session get
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/ExtendedNiFiRegistryClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/ExtendedNiFiRegistryClient.java
new file mode 100644
index 0000000..cecce2e
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/ExtendedNiFiRegistryClient.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.toolkit.cli.impl.client;
+
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.PoliciesClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.TenantsClient;
+
+/**
+ * Extends NiFiRegistry client with additional exposed service.
+ *
+ * Note: in longer term the functionality of this should be merged into the NiFiRegistryClient.
+ */
+public interface ExtendedNiFiRegistryClient extends NiFiRegistryClient {
+
+    /**
+     * 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);
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/JerseyExtendedNiFiRegistryClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/JerseyExtendedNiFiRegistryClient.java
new file mode 100644
index 0000000..edf0300
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/JerseyExtendedNiFiRegistryClient.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.toolkit.cli.impl.client;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.databind.DeserializationFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.fasterxml.jackson.module.jaxb.JaxbAnnotationIntrospector;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.bucket.BucketItem;
+import org.apache.nifi.registry.client.BucketClient;
+import org.apache.nifi.registry.client.BundleClient;
+import org.apache.nifi.registry.client.BundleVersionClient;
+import org.apache.nifi.registry.client.ExtensionClient;
+import org.apache.nifi.registry.client.ExtensionRepoClient;
+import org.apache.nifi.registry.client.FlowClient;
+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.UserClient;
+import org.apache.nifi.registry.client.impl.BucketItemDeserializer;
+import org.apache.nifi.registry.security.util.ProxiedEntitiesUtils;
+import org.apache.nifi.toolkit.cli.impl.client.registry.PoliciesClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.TenantsClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.impl.JerseyPoliciesClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.impl.JerseyTenantsClient;
+import org.glassfish.jersey.client.ClientConfig;
+import org.glassfish.jersey.client.ClientProperties;
+import org.glassfish.jersey.client.RequestEntityProcessing;
+import org.glassfish.jersey.jackson.internal.jackson.jaxrs.json.JacksonJaxbJsonProvider;
+import org.glassfish.jersey.media.multipart.MultiPartFeature;
+
+import javax.net.ssl.HostnameVerifier;
+import javax.net.ssl.SSLContext;
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+import javax.ws.rs.client.WebTarget;
+import java.io.IOException;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/**
+ * Decorator around (Jersey based) NiFiRegistryClient in order to extend it's capabilities without
+ * actually changing it.
+ */
+public class JerseyExtendedNiFiRegistryClient implements ExtendedNiFiRegistryClient {
+    // Copied from JerseyNiFiRegistryClient!
+    static final String NIFI_REGISTRY_CONTEXT = "nifi-registry-api";
+    static final int DEFAULT_CONNECT_TIMEOUT = 10000;
+    static final int DEFAULT_READ_TIMEOUT = 10000;
+
+    private final NiFiRegistryClient payload;
+    private final Client client;
+    private final WebTarget baseTarget;
+    private final TenantsClient tenantsClient;
+    private final PoliciesClient policiesClient;
+
+    public JerseyExtendedNiFiRegistryClient(final NiFiRegistryClient payload, final NiFiRegistryClient.Builder builder) {
+        this.payload = payload;
+
+        // Copied from JerseyNiFiRegistryClient!
+        final NiFiRegistryClientConfig registryClientConfig = builder.getConfig();
+        if (registryClientConfig == null) {
+            throw new IllegalArgumentException("NiFiRegistryClientConfig cannot be null");
+        }
+
+        String baseUrl = registryClientConfig.getBaseUrl();
+        if (StringUtils.isBlank(baseUrl)) {
+            throw new IllegalArgumentException("Base URL cannot be blank");
+        }
+
+        if (baseUrl.endsWith("/")) {
+            baseUrl = baseUrl.substring(0, baseUrl.length() - 1);
+        }
+
+        if (!baseUrl.endsWith(NIFI_REGISTRY_CONTEXT)) {
+            baseUrl = baseUrl + "/" + NIFI_REGISTRY_CONTEXT;
+        }
+
+        try {
+            new URI(baseUrl);
+        } catch (final Exception e) {
+            throw new IllegalArgumentException("Invalid base URL: " + e.getMessage(), e);
+        }
+
+        final SSLContext sslContext = registryClientConfig.getSslContext();
+        final HostnameVerifier hostnameVerifier = registryClientConfig.getHostnameVerifier();
+
+        final ClientBuilder clientBuilder = ClientBuilder.newBuilder();
+        if (sslContext != null) {
+            clientBuilder.sslContext(sslContext);
+        }
+        if (hostnameVerifier != null) {
+            clientBuilder.hostnameVerifier(hostnameVerifier);
+        }
+
+        final int connectTimeout = registryClientConfig.getConnectTimeout() == null ? DEFAULT_CONNECT_TIMEOUT : registryClientConfig.getConnectTimeout();
+        final int readTimeout = registryClientConfig.getReadTimeout() == null ? DEFAULT_READ_TIMEOUT : registryClientConfig.getReadTimeout();
+
+        final ClientConfig clientConfig = new ClientConfig();
+        clientConfig.property(ClientProperties.CONNECT_TIMEOUT, connectTimeout);
+        clientConfig.property(ClientProperties.READ_TIMEOUT, readTimeout);
+        clientConfig.property(ClientProperties.REQUEST_ENTITY_PROCESSING, RequestEntityProcessing.CHUNKED);
+        clientConfig.register(jacksonJaxbJsonProvider());
+        clientBuilder.withConfig(clientConfig);
+
+        this.client = clientBuilder
+                .register(MultiPartFeature.class)
+                .build();
+
+        this.baseTarget = client.target(baseUrl);
+
+        this.tenantsClient = new JerseyTenantsClient(baseTarget);
+        this.policiesClient = new JerseyPoliciesClient(baseTarget);
+    }
+
+    @Override
+    public TenantsClient getTenantsClient() {
+        return tenantsClient;
+    }
+
+    @Override
+    public TenantsClient getTenantsClient(final String... proxiedEntity) {
+        final Map<String,String> headers = getHeaders(proxiedEntity);
+        return new JerseyTenantsClient(baseTarget, headers);
+    }
+
+    @Override
+    public PoliciesClient getPoliciesClient() {
+        return policiesClient;
+    }
+
+    @Override
+    public PoliciesClient getPoliciesClient(final String... proxiedEntity) {
+        final Map<String,String> headers = getHeaders(proxiedEntity);
+        return new JerseyPoliciesClient(baseTarget, headers);
+    }
+
+    @Override
+    public BucketClient getBucketClient() {
+        return payload.getBucketClient();
+    }
+
+    @Override
+    public BucketClient getBucketClient(final String... proxiedEntity) {
+        return payload.getBucketClient(proxiedEntity);
+    }
+
+    @Override
+    public FlowClient getFlowClient() {
+        return payload.getFlowClient();
+    }
+
+    @Override
+    public FlowClient getFlowClient(final String... proxiedEntity) {
+        return payload.getFlowClient(proxiedEntity);
+    }
+
+    @Override
+    public FlowSnapshotClient getFlowSnapshotClient() {
+        return payload.getFlowSnapshotClient();
+    }
+
+    @Override
+    public FlowSnapshotClient getFlowSnapshotClient(final String... proxiedEntity) {
+        return payload.getFlowSnapshotClient(proxiedEntity);
+    }
+
+    @Override
+    public ItemsClient getItemsClient() {
+        return payload.getItemsClient();
+    }
+
+    @Override
+    public ItemsClient getItemsClient(final String... proxiedEntity) {
+        return payload.getItemsClient(proxiedEntity);
+    }
+
+    @Override
+    public UserClient getUserClient() {
+        return payload.getUserClient();
+    }
+
+    @Override
+    public UserClient getUserClient(final String... proxiedEntity) {
+        return payload.getUserClient(proxiedEntity);
+    }
+
+    @Override
+    public BundleClient getBundleClient() {
+        return payload.getBundleClient();
+    }
+
+    @Override
+    public BundleClient getBundleClient(final String... proxiedEntity) {
+        return payload.getBundleClient(proxiedEntity);
+    }
+
+    @Override
+    public BundleVersionClient getBundleVersionClient() {
+        return payload.getBundleVersionClient();
+    }
+
+    @Override
+    public BundleVersionClient getBundleVersionClient(final String... proxiedEntity) {
+        return payload.getBundleVersionClient(proxiedEntity);
+    }
+
+    @Override
+    public ExtensionRepoClient getExtensionRepoClient() {
+        return payload.getExtensionRepoClient();
+    }
+
+    @Override
+    public ExtensionRepoClient getExtensionRepoClient(final String... proxiedEntity) {
+        return payload.getExtensionRepoClient(proxiedEntity);
+    }
+
+    @Override
+    public ExtensionClient getExtensionClient() {
+        return payload.getExtensionClient();
+    }
+
+    @Override
+    public ExtensionClient getExtensionClient(final String... proxiedEntity) {
+        return payload.getExtensionClient(proxiedEntity);
+    }
+
+    @Override
+    public void close() throws IOException {
+        payload.close();
+
+        if (this.client != null) {
+            try {
+                this.client.close();
+            } catch (Exception e) {
+
+            }
+        }
+    }
+
+    // Copied from JerseyNiFiRegistryClient!
+    private Map<String,String> getHeaders(String[] proxiedEntities) {
+        final String proxiedEntitiesValue = getProxiedEntitesValue(proxiedEntities);
+
+        final Map<String,String> headers = new HashMap<>();
+        if (proxiedEntitiesValue != null) {
+            headers.put(ProxiedEntitiesUtils.PROXY_ENTITIES_CHAIN, proxiedEntitiesValue);
+        }
+        return headers;
+    }
+
+    // Copied from JerseyNiFiRegistryClient!
+    private String getProxiedEntitesValue(String[] proxiedEntities) {
+        if (proxiedEntities == null) {
+            return null;
+        }
+
+        final List<String> proxiedEntityChain = Arrays.stream(proxiedEntities).map(ProxiedEntitiesUtils::formatProxyDn).collect(Collectors.toList());
+        return StringUtils.join(proxiedEntityChain, "");
+    }
+
+    // Copied from JerseyNiFiRegistryClient!
+    private static JacksonJaxbJsonProvider jacksonJaxbJsonProvider() {
+        JacksonJaxbJsonProvider jacksonJaxbJsonProvider = new JacksonJaxbJsonProvider();
+
+        ObjectMapper mapper = new ObjectMapper();
+        mapper.setPropertyInclusion(JsonInclude.Value.construct(JsonInclude.Include.NON_NULL, JsonInclude.Include.NON_NULL));
+        mapper.setAnnotationIntrospector(new JaxbAnnotationIntrospector(mapper.getTypeFactory()));
+        // Ignore unknown properties so that deployed client remain compatible with future versions of NiFi Registry that add new fields
+        mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
+
+        SimpleModule module = new SimpleModule();
+        module.addDeserializer(BucketItem[].class, new BucketItemDeserializer());
+        mapper.registerModule(module);
+
+        jacksonJaxbJsonProvider.setMapper(mapper);
+        return jacksonJaxbJsonProvider;
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiRegistryClientFactory.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiRegistryClientFactory.java
index bdc2535..cbdfd36 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiRegistryClientFactory.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/NiFiRegistryClientFactory.java
@@ -32,6 +32,8 @@ import org.apache.nifi.registry.client.UserClient;
 import org.apache.nifi.registry.client.impl.JerseyNiFiRegistryClient;
 import org.apache.nifi.registry.security.util.KeystoreType;
 import org.apache.nifi.toolkit.cli.api.ClientFactory;
+import org.apache.nifi.toolkit.cli.impl.client.registry.PoliciesClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.TenantsClient;
 import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
 
 import java.io.IOException;
@@ -97,13 +99,15 @@ public class NiFiRegistryClientFactory implements ClientFactory<NiFiRegistryClie
             }
         }
 
-        final NiFiRegistryClient client = new JerseyNiFiRegistryClient.Builder().config(clientConfigBuilder.build()).build();
+        final NiFiRegistryClientConfig builder = clientConfigBuilder.build();
+        final NiFiRegistryClient client = new JerseyNiFiRegistryClient.Builder().config(builder).build();
+        final ExtendedNiFiRegistryClient extendedClient = new JerseyExtendedNiFiRegistryClient(client, new JerseyNiFiRegistryClient.Builder().config(builder));
 
         // if a proxied entity was specified then return a wrapped client, otherwise return the regular client
         if (!StringUtils.isBlank(proxiedEntity)) {
-            return new ProxiedNiFiRegistryClient(client, proxiedEntity);
+            return new ProxiedNiFiRegistryClient(extendedClient, proxiedEntity);
         } else {
-            return client;
+            return extendedClient;
         }
     }
 
@@ -111,12 +115,12 @@ public class NiFiRegistryClientFactory implements ClientFactory<NiFiRegistryClie
      * Wraps a NiFiRegistryClient and ensures that all methods to obtain a more specific client will
      * call the proxied-entity variation so that callers don't have to care if proxying is taking place.
      */
-    private static class ProxiedNiFiRegistryClient implements NiFiRegistryClient {
+    private static class ProxiedNiFiRegistryClient implements ExtendedNiFiRegistryClient {
 
-        private final NiFiRegistryClient client;
+        private final ExtendedNiFiRegistryClient client;
         private final String proxiedEntity;
 
-        public ProxiedNiFiRegistryClient(final NiFiRegistryClient client, final String proxiedEntity) {
+        public ProxiedNiFiRegistryClient(final ExtendedNiFiRegistryClient client, final String proxiedEntity) {
             this.client = client;
             this.proxiedEntity = proxiedEntity;
         }
@@ -212,9 +216,28 @@ public class NiFiRegistryClientFactory implements ClientFactory<NiFiRegistryClie
         }
 
         @Override
+        public TenantsClient getTenantsClient() {
+            return getTenantsClient(proxiedEntity);
+        }
+
+        @Override
+        public TenantsClient getTenantsClient(String... proxiedEntity) {
+            return client.getTenantsClient(proxiedEntity);
+        }
+
+        @Override
+        public PoliciesClient getPoliciesClient() {
+            return getPoliciesClient(proxiedEntity);
+        }
+
+        @Override
+        public PoliciesClient getPoliciesClient(String... proxiedEntity) {
+            return client.getPoliciesClient(proxiedEntity);
+        }
+
+        @Override
         public void close() throws IOException {
             client.close();
         }
     }
-
 }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/PoliciesClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/PoliciesClient.java
new file mode 100644
index 0000000..be481a8
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/PoliciesClient.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.toolkit.cli.impl.client.registry;
+
+import org.apache.nifi.registry.authorization.AccessPolicy;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+
+import java.io.IOException;
+
+/**
+ * Provides API for the services might be called from registry related to access policies.
+ */
+public interface PoliciesClient {
+
+    /**
+     * Returns with a given access policy.
+     *
+     * @param id The identifier of the access policy.
+     *
+     * @return The access policy.
+     *
+     * @throws NiFiRegistryException Thrown in case os unsuccessful execution.
+     * @throws IOException Thrown when there is an issue with communicating the registry.
+     */
+    AccessPolicy getPolicy(String id) throws NiFiRegistryException, IOException;
+
+    /**
+     * Creates a new access policy within the registry.
+     *
+     * @param policy The attributes of the access policy. Note: identifier will be ignored and generated.
+     *
+     * @return The access policy after store, containing it's identifier.
+     *
+     * @throws NiFiRegistryException Thrown in case os unsuccessful execution.
+     * @throws IOException Thrown when there is an issue with communicating the registry.
+     */
+    AccessPolicy createPolicy(AccessPolicy policy) throws NiFiRegistryException, IOException;
+
+    /**
+     * Updates an existing access policy.
+     *
+     * @param policy The updated attributes of the access policy.
+     *
+     * @return The stored access policy.
+     *
+     * @throws NiFiRegistryException Thrown in case os unsuccessful execution.
+     * @throws IOException Thrown when there is an issue with communicating the registry.
+     */
+    AccessPolicy updatePolicy(AccessPolicy policy) throws NiFiRegistryException, IOException;
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/TenantsClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/TenantsClient.java
new file mode 100644
index 0000000..5ad56f8
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/TenantsClient.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.toolkit.cli.impl.client.registry;
+
+import org.apache.nifi.registry.authorization.User;
+import org.apache.nifi.registry.authorization.UserGroup;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * Provides API for the services might be called from registry related to tenants.
+ */
+public interface TenantsClient {
+
+    /**
+     * Returns all the users.
+     *
+     * @return The list of users in the registry.
+     *
+     * @throws NiFiRegistryException Thrown in case os unsuccessful execution.
+     * @throws IOException Thrown when there is an issue with communicating the registry.
+     */
+    List<User> getUsers() throws NiFiRegistryException, IOException;
+
+    /**
+     * Returns a given user based on id.
+     *
+     * @param id Identifier of the user.
+     *
+     * @return The user.
+     *
+     * @throws NiFiRegistryException Thrown in case os unsuccessful execution.
+     * @throws IOException Thrown when there is an issue with communicating the registry.
+     */
+    User getUser(String id) throws NiFiRegistryException, IOException;
+
+    /**
+     * Creates a new user in the registry.
+     *
+     * @param user The new user. Note: identifier will be ignored and generated.
+     *
+     * @return The user after store, containing it's identifier.
+     *
+     * @throws NiFiRegistryException Thrown in case os unsuccessful execution.
+     * @throws IOException Thrown when there is an issue with communicating the registry.
+     */
+    User createUser(User user) throws NiFiRegistryException, IOException;
+
+    /**
+     * Updates an existing user.
+     *
+     * @param user The user with the new attributes.
+     *
+     * @return The user after store.
+     *
+     * @throws NiFiRegistryException Thrown in case os unsuccessful execution.
+     * @throws IOException Thrown when there is an issue with communicating the registry.
+     */
+    User updateUser(User user) throws NiFiRegistryException, IOException;
+
+    /**
+     * Returns all the user groups.
+     *
+     * @return The list of user groups.
+     *
+     * @throws NiFiRegistryException Thrown in case os unsuccessful execution.
+     * @throws IOException Thrown when there is an issue with communicating the registry.
+     */
+    List<UserGroup> getUserGroups() throws NiFiRegistryException, IOException;
+
+    /**
+     * Returns the given user group based on identifier.
+     *
+     * @param id The user group's identifier.
+     *
+     * @return The user group.
+     *
+     * @throws NiFiRegistryException Thrown in case os unsuccessful execution.
+     * @throws IOException Thrown when there is an issue with communicating the registry.
+     */
+    UserGroup getUserGroup(String id) throws NiFiRegistryException, IOException;
+
+    /**
+     * Creates a new user group in the registry.
+     *
+     * @param group The user group to store. Note: identifier will be ignored and generated.
+     *
+     * @return The stored user group, containing id.
+     *
+     * @throws NiFiRegistryException Thrown in case os unsuccessful execution.
+     * @throws IOException Thrown when there is an issue with communicating the registry.@throws IOException
+     */
+    UserGroup createUserGroup(UserGroup group) throws NiFiRegistryException, IOException;
+
+    /**
+     * Updates an existing user group.
+     *
+     * @param group The user group with the new attributes.
+     *
+     * @return The user group after store.
+     *
+     * @throws NiFiRegistryException Thrown in case os unsuccessful execution.
+     * @throws IOException Thrown when there is an issue with communicating the registry.
+     */
+    UserGroup updateUserGroup(UserGroup group) throws NiFiRegistryException, IOException;
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/impl/JerseyPoliciesClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/impl/JerseyPoliciesClient.java
new file mode 100644
index 0000000..ee7a806
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/impl/JerseyPoliciesClient.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.toolkit.cli.impl.client.registry.impl;
+
+import org.apache.nifi.registry.authorization.AccessPolicy;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.registry.client.impl.AbstractJerseyClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.PoliciesClient;
+import org.apache.nifi.util.StringUtils;
+
+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.Collections;
+import java.util.Map;
+
+public class JerseyPoliciesClient extends AbstractJerseyClient implements PoliciesClient {
+    private final WebTarget policiesTarget;
+
+    public JerseyPoliciesClient(final WebTarget baseTarget, final Map<String, String> headers) {
+        super(headers);
+        this.policiesTarget = baseTarget.path("/policies");
+    }
+
+    public JerseyPoliciesClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    @Override
+    public AccessPolicy getPolicy(final String id) throws NiFiRegistryException, IOException {
+        if (StringUtils.isBlank(id)) {
+            throw new IllegalArgumentException("Access policy id cannot be null");
+        }
+
+        return executeAction("Error retrieving access policy", () -> {
+            final WebTarget target = policiesTarget.path("{id}").resolveTemplate("id", id);
+            return getRequestBuilder(target).get(AccessPolicy.class);
+        });
+    }
+
+    @Override
+    public AccessPolicy createPolicy(final AccessPolicy policy) throws NiFiRegistryException, IOException {
+        if (policy == null) {
+            throw new IllegalArgumentException("Access policy cannot be null");
+        }
+
+        return executeAction("Error creating access policy", () -> {
+            return getRequestBuilder(policiesTarget).post(
+                    Entity.entity(policy, MediaType.APPLICATION_JSON_TYPE), AccessPolicy.class
+            );
+        });
+    }
+
+    @Override
+    public AccessPolicy updatePolicy(final AccessPolicy policy) throws NiFiRegistryException, IOException {
+        if (policy == null) {
+            throw new IllegalArgumentException("Access policy cannot be null");
+        }
+
+        return executeAction("Error creating access policy", () -> {
+            final WebTarget target = policiesTarget.path("{id}").resolveTemplate("id", policy.getIdentifier());
+            return getRequestBuilder(target).put(
+                    Entity.entity(policy, MediaType.APPLICATION_JSON_TYPE), AccessPolicy.class
+            );
+        });
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/impl/JerseyTenantsClient.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/impl/JerseyTenantsClient.java
new file mode 100644
index 0000000..a135702
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/client/registry/impl/JerseyTenantsClient.java
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.toolkit.cli.impl.client.registry.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.impl.AbstractJerseyClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.TenantsClient;
+import org.apache.nifi.util.StringUtils;
+
+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.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class JerseyTenantsClient extends AbstractJerseyClient implements TenantsClient {
+    private final WebTarget tenantsTarget;
+
+    public JerseyTenantsClient(final WebTarget baseTarget, final Map<String, String> headers) {
+        super(headers);
+        this.tenantsTarget = baseTarget.path("/tenants");
+    }
+
+    public JerseyTenantsClient(final WebTarget baseTarget) {
+        this(baseTarget, Collections.emptyMap());
+    }
+
+    @Override
+    public List<User> getUsers() throws NiFiRegistryException, IOException {
+        return executeAction("Error retrieving users", () -> {
+            final WebTarget target = tenantsTarget.path("users");
+            return Arrays.asList(getRequestBuilder(target).get(User[].class));
+        });
+    }
+
+    @Override
+    public User getUser(final String id) throws NiFiRegistryException, IOException {
+        if (StringUtils.isBlank(id)) {
+            throw new IllegalArgumentException("User id cannot be null");
+        }
+
+        return executeAction("Error retrieving user", () -> {
+            final WebTarget target = tenantsTarget.path("users/{id}").resolveTemplate("id", id);
+            return getRequestBuilder(target).get(User.class);
+        });
+    }
+
+    @Override
+    public User createUser(final User user) throws NiFiRegistryException, IOException {
+        if (user == null) {
+            throw new IllegalArgumentException("User cannot be null");
+        }
+
+        return executeAction("Error creating user", () -> {
+            final WebTarget target = tenantsTarget.path("users");
+
+            return getRequestBuilder(target).post(
+                Entity.entity(user, MediaType.APPLICATION_JSON_TYPE), User.class
+            );
+        });
+    }
+
+    @Override
+    public User updateUser(final User user) throws NiFiRegistryException, IOException {
+        if (user == null) {
+            throw new IllegalArgumentException("User cannot be null");
+        }
+
+        return executeAction("Error updating user", () -> {
+            final WebTarget target = tenantsTarget.path("users/{id}").resolveTemplate("id", user.getIdentifier());
+
+            return getRequestBuilder(target).put(
+                    Entity.entity(user, MediaType.APPLICATION_JSON_TYPE), User.class
+            );
+        });
+    }
+
+    @Override
+    public List<UserGroup> getUserGroups() throws NiFiRegistryException, IOException {
+        return executeAction("Error retrieving users", () -> {
+            final WebTarget target = tenantsTarget.path("user-groups");
+            return Arrays.asList(getRequestBuilder(target).get(UserGroup[].class));
+        });
+    }
+
+    @Override
+    public UserGroup getUserGroup(final String id) throws NiFiRegistryException, IOException {
+        if (StringUtils.isBlank(id)) {
+            throw new IllegalArgumentException("User group id cannot be null");
+        }
+
+        return executeAction("Error retrieving user group", () -> {
+            final WebTarget target = tenantsTarget.path("user-groups/{id}").resolveTemplate("id", id);
+            return getRequestBuilder(target).get(UserGroup.class);
+        });
+    }
+
+    @Override
+    public UserGroup createUserGroup(final UserGroup group) throws NiFiRegistryException, IOException {
+        if (group == null) {
+            throw new IllegalArgumentException("User group cannot be null");
+        }
+
+        return executeAction("Error creating group", () -> {
+            final WebTarget target = tenantsTarget.path("user-groups");
+
+            return getRequestBuilder(target).post(
+                    Entity.entity(group, MediaType.APPLICATION_JSON_TYPE), UserGroup.class
+            );
+        });
+    }
+
+    @Override
+    public UserGroup updateUserGroup(final UserGroup group) throws NiFiRegistryException, IOException {
+        if (group == null) {
+            throw new IllegalArgumentException("User group cannot be null");
+        }
+
+        return executeAction("Error creating group", () -> {
+            final WebTarget target = tenantsTarget.path("user-groups/{id}").resolveTemplate("id", group.getIdentifier());
+
+            return getRequestBuilder(target).put(
+                    Entity.entity(group, MediaType.APPLICATION_JSON_TYPE), UserGroup.class
+            );
+        });
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java
index 623d555..eb5b28d 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/CommandOption.java
@@ -93,6 +93,7 @@ public enum CommandOption {
 
     // NiFi - User/Group
     USER_NAME("un", "userName", "The name of a user", true),
+    USER_ID("ui", "userIdentifier", "The identifier of a user", true),
     UG_ID("ugid", "userGroupId", "The id of a user group", true),
     UG_NAME("ugn", "userGroupName", "The name of a user group", true),
     USER_NAME_LIST("unl", "userNameList", "The comma-separated user name list", true),
@@ -101,6 +102,7 @@ public enum CommandOption {
     GROUP_ID_LIST("gil", "groupIdList", "The comma-separated user group id list", true),
 
     // NiFi - Access Policies
+    POLICY_ID("pi", "accessPolicyIdentifier", "The identifier of an access policy", true),
     POLICY_RESOURCE("por", "accessPolicyResource", "The resource of an access policy", true),
     POLICY_ACTION("poa", "accessPolicyAction", "The action of an access policy (read or write)", true),
     OVERWRITE_POLICY("owp", "overwritePolicy", "Overwrite the user list and group list for the access policy", false),
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/NiFiRegistryCommandGroup.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/NiFiRegistryCommandGroup.java
index 3e14906..8b9c93f 100644
--- a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/NiFiRegistryCommandGroup.java
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/NiFiRegistryCommandGroup.java
@@ -39,6 +39,12 @@ import org.apache.nifi.toolkit.cli.impl.command.registry.flow.ListFlowVersions;
 import org.apache.nifi.toolkit.cli.impl.command.registry.flow.ListFlows;
 import org.apache.nifi.toolkit.cli.impl.command.registry.flow.SyncFlowVersions;
 import org.apache.nifi.toolkit.cli.impl.command.registry.flow.TransferFlowVersion;
+import org.apache.nifi.toolkit.cli.impl.command.registry.tenant.CreatePolicy;
+import org.apache.nifi.toolkit.cli.impl.command.registry.tenant.CreateUser;
+import org.apache.nifi.toolkit.cli.impl.command.registry.tenant.CreateUserGroup;
+import org.apache.nifi.toolkit.cli.impl.command.registry.tenant.UpdatePolicy;
+import org.apache.nifi.toolkit.cli.impl.command.registry.tenant.UpdateUser;
+import org.apache.nifi.toolkit.cli.impl.command.registry.tenant.UpdateUserGroup;
 import org.apache.nifi.toolkit.cli.impl.command.registry.user.CurrentUser;
 
 import java.util.ArrayList;
@@ -80,6 +86,12 @@ public class NiFiRegistryCommandGroup extends AbstractCommandGroup {
         commandList.add(new GetBundleChecksum());
         commandList.add(new ListExtensionTags());
         commandList.add(new ListExtensions());
+        commandList.add(new CreateUser());
+        commandList.add(new UpdateUser());
+        commandList.add(new CreateUserGroup());
+        commandList.add(new UpdateUserGroup());
+        commandList.add(new CreatePolicy());
+        commandList.add(new UpdatePolicy());
         return new ArrayList<>(commandList);
     }
 }
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/CreatePolicy.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/CreatePolicy.java
new file mode 100644
index 0000000..5ab45aa
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/CreatePolicy.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.toolkit.cli.impl.command.registry.tenant;
+
+import org.apache.commons.cli.ParseException;
+import org.apache.nifi.registry.authorization.AccessPolicy;
+import org.apache.nifi.registry.authorization.Tenant;
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.toolkit.cli.api.Context;
+import org.apache.nifi.toolkit.cli.impl.client.ExtendedNiFiRegistryClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.PoliciesClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.TenantsClient;
+import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
+import org.apache.nifi.toolkit.cli.impl.command.registry.AbstractNiFiRegistryCommand;
+import org.apache.nifi.toolkit.cli.impl.result.StringResult;
+
+import java.io.IOException;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Command for creating an access policy.
+ */
+public class CreatePolicy extends AbstractNiFiRegistryCommand<StringResult> {
+
+    public CreatePolicy() {
+        super("create-policy", StringResult.class);
+    }
+
+    @Override
+    public String getDescription() {
+        return "Creates new access policy";
+    }
+
+    @Override
+    protected void doInitialize(final Context context) {
+        // Required
+        addOption(CommandOption.POLICY_RESOURCE.createOption());
+        addOption(CommandOption.POLICY_ACTION.createOption());
+
+        // Optional
+        addOption(CommandOption.USER_NAME_LIST.createOption());
+        addOption(CommandOption.USER_ID_LIST.createOption());
+        addOption(CommandOption.GROUP_NAME_LIST.createOption());
+        addOption(CommandOption.GROUP_ID_LIST.createOption());
+    }
+
+    @Override
+    public StringResult doExecute(final NiFiRegistryClient client, final Properties properties)
+            throws IOException, NiFiRegistryException, ParseException {
+
+        if (!(client instanceof ExtendedNiFiRegistryClient)) {
+            throw new IllegalArgumentException("This command needs extended registry client!");
+        }
+
+        final ExtendedNiFiRegistryClient extendedClient = (ExtendedNiFiRegistryClient) client;
+        final PoliciesClient policiesClient = extendedClient.getPoliciesClient();
+        final TenantsClient tenantsClient = extendedClient.getTenantsClient();
+
+        final String resource = getRequiredArg(properties, CommandOption.POLICY_RESOURCE);
+        final String action = getRequiredArg(properties, CommandOption.POLICY_ACTION);
+
+        final Set<Tenant> users = TenantHelper.getExistingUsers(
+                tenantsClient,
+                getArg(properties, CommandOption.USER_NAME_LIST),
+                getArg(properties, CommandOption.USER_ID_LIST));
+
+        final Set<Tenant> userGroups = TenantHelper.getExistingGroups(
+                tenantsClient,
+                getArg(properties, CommandOption.GROUP_NAME_LIST),
+                getArg(properties, CommandOption.GROUP_ID_LIST));
+
+        final AccessPolicy policy = new AccessPolicy();
+        policy.setAction(action);
+        policy.setResource(resource);
+        policy.setUsers(users);
+        policy.setUserGroups(userGroups);
+
+        final AccessPolicy createdPolicy = policiesClient.createPolicy(policy);
+        return new StringResult(createdPolicy.getIdentifier(), getContext().isInteractive());
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/CreateUser.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/CreateUser.java
new file mode 100644
index 0000000..baf61f0
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/CreateUser.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.toolkit.cli.impl.command.registry.tenant;
+
+import org.apache.commons.cli.ParseException;
+import org.apache.nifi.registry.authorization.User;
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.toolkit.cli.api.Context;
+import org.apache.nifi.toolkit.cli.impl.client.ExtendedNiFiRegistryClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.TenantsClient;
+import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
+import org.apache.nifi.toolkit.cli.impl.command.registry.AbstractNiFiRegistryCommand;
+import org.apache.nifi.toolkit.cli.impl.result.StringResult;
+
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * Command for creating a user.
+ */
+public class CreateUser extends AbstractNiFiRegistryCommand<StringResult> {
+
+    public CreateUser() {
+        super("create-user", StringResult.class);
+    }
+
+    @Override
+    public String getDescription() {
+        return "Creates new user.";
+    }
+
+    @Override
+    protected void doInitialize(final Context context) {
+        addOption(CommandOption.USER_NAME.createOption());
+    }
+
+    @Override
+    public StringResult doExecute(final NiFiRegistryClient client, final Properties properties)
+            throws IOException, NiFiRegistryException, ParseException {
+
+        if (!(client instanceof ExtendedNiFiRegistryClient)) {
+            throw new IllegalArgumentException("This command needs extended registry client!");
+        }
+
+        final ExtendedNiFiRegistryClient extendedClient = (ExtendedNiFiRegistryClient) client;
+        final TenantsClient tenantsClient = extendedClient.getTenantsClient();
+
+        final String userName = getRequiredArg(properties, CommandOption.USER_NAME);
+        final User user = new User(null, userName);
+        final User createdUser = tenantsClient.createUser(user);
+
+        return new StringResult(createdUser.getIdentifier(), getContext().isInteractive());
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/CreateUserGroup.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/CreateUserGroup.java
new file mode 100644
index 0000000..fced142
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/CreateUserGroup.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.toolkit.cli.impl.command.registry.tenant;
+
+import org.apache.commons.cli.ParseException;
+import org.apache.nifi.registry.authorization.Tenant;
+import org.apache.nifi.registry.authorization.UserGroup;
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.toolkit.cli.api.Context;
+import org.apache.nifi.toolkit.cli.impl.client.ExtendedNiFiRegistryClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.TenantsClient;
+import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
+import org.apache.nifi.toolkit.cli.impl.command.registry.AbstractNiFiRegistryCommand;
+import org.apache.nifi.toolkit.cli.impl.result.StringResult;
+
+import java.io.IOException;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Command for creating a user group.
+ */
+public class CreateUserGroup extends AbstractNiFiRegistryCommand<StringResult> {
+
+    public CreateUserGroup() {
+        super("create-user-group", StringResult.class);
+    }
+
+    @Override
+    public String getDescription() {
+        return "Creates user group";
+    }
+
+    @Override
+    protected void doInitialize(final Context context) {
+        // Required
+        addOption(CommandOption.UG_NAME.createOption());
+
+        // Optional
+        addOption(CommandOption.USER_NAME_LIST.createOption());
+        addOption(CommandOption.USER_ID_LIST.createOption());
+    }
+
+    @Override
+    public StringResult doExecute(final NiFiRegistryClient client, final Properties properties)
+            throws IOException, NiFiRegistryException, ParseException {
+
+        if (!(client instanceof ExtendedNiFiRegistryClient)) {
+            throw new IllegalArgumentException("This command needs extended registry client!");
+        }
+
+        final ExtendedNiFiRegistryClient extendedClient = (ExtendedNiFiRegistryClient) client;
+        final TenantsClient tenantsClient = extendedClient.getTenantsClient();
+
+        final String groupName = getRequiredArg(properties, CommandOption.UG_NAME);
+
+        final Set<Tenant> tenants = TenantHelper.getExistingUsers(
+                tenantsClient,
+                getArg(properties, CommandOption.USER_NAME_LIST),
+                getArg(properties, CommandOption.USER_ID_LIST));
+
+        final UserGroup group = new UserGroup(null, groupName);
+        group.setUsers(tenants);
+
+        final UserGroup createdGroup = tenantsClient.createUserGroup(group);
+        return new StringResult(createdGroup.getIdentifier(), getContext().isInteractive());
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/TenantHelper.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/TenantHelper.java
new file mode 100644
index 0000000..64415f9
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/TenantHelper.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.toolkit.cli.impl.command.registry.tenant;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.authorization.Tenant;
+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.toolkit.cli.impl.client.registry.TenantsClient;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+final class TenantHelper {
+    private static final String SEPARATOR = ",";
+
+    private TenantHelper() {
+        // no op
+    }
+
+    public static Set<Tenant> getExistingUsers(final TenantsClient client, final String userNamesArgument, final String userIdsArgument)
+            throws IOException, NiFiRegistryException {
+        final Set<Tenant> result = new HashSet<>();
+
+        final Set<String> userNames = StringUtils.isNotBlank(userNamesArgument)
+                ? new HashSet<>(Arrays.asList(userNamesArgument.split(SEPARATOR)))
+                : Collections.emptySet();
+
+        final Set<String> userIds = StringUtils.isNotBlank(userIdsArgument)
+                ? new HashSet<>(Arrays.asList(userIdsArgument.split(SEPARATOR)))
+                : Collections.emptySet();
+
+        if (userNames.isEmpty() && userIds.isEmpty()) {
+            return result;
+        }
+
+        final List<User> users = client.getUsers();
+
+        for (final User user : users) {
+            if (userNames.contains(user.getIdentity()) || userIds.contains(user.getIdentifier())) {
+                result.add(user);
+            }
+        }
+
+        return result;
+    }
+
+    public static Set<Tenant> getExistingGroups(final TenantsClient client, final String userGroupNamesArgument, final String userGroupIdsArgument)
+            throws IOException, NiFiRegistryException {
+        final Set<Tenant> result = new HashSet<>();
+
+        final Set<String> userGroupNames = StringUtils.isNotBlank(userGroupNamesArgument)
+                ? new HashSet<>(Arrays.asList(userGroupNamesArgument.split(SEPARATOR)))
+                : Collections.emptySet();
+
+        final Set<String> userGroupIds = StringUtils.isNotBlank(userGroupIdsArgument)
+                ? new HashSet<>(Arrays.asList(userGroupIdsArgument.split(SEPARATOR)))
+                : Collections.emptySet();
+
+        if (userGroupNames.isEmpty() && userGroupIds.isEmpty()) {
+            return result;
+        }
+
+        final List<UserGroup> usersGroups = client.getUserGroups();
+
+        for (final UserGroup userGroup : usersGroups) {
+            if (userGroupNames.contains(userGroup.getIdentity()) || userGroupIds.contains(userGroup.getIdentifier())) {
+                result.add(userGroup);
+            }
+        }
+
+        return result;
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/UpdatePolicy.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/UpdatePolicy.java
new file mode 100644
index 0000000..f30f3cf
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/UpdatePolicy.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.toolkit.cli.impl.command.registry.tenant;
+
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.authorization.AccessPolicy;
+import org.apache.nifi.registry.authorization.Tenant;
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.toolkit.cli.api.Context;
+import org.apache.nifi.toolkit.cli.impl.client.ExtendedNiFiRegistryClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.PoliciesClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.TenantsClient;
+import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
+import org.apache.nifi.toolkit.cli.impl.command.registry.AbstractNiFiRegistryCommand;
+import org.apache.nifi.toolkit.cli.impl.result.StringResult;
+
+import java.io.IOException;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Command for update an existing policy.
+ */
+public class UpdatePolicy extends AbstractNiFiRegistryCommand<StringResult> {
+
+    public UpdatePolicy() {
+        super("update-policy", StringResult.class);
+    }
+
+    @Override
+    public String getDescription() {
+        return "Updates an existing access policy.";
+    }
+
+    @Override
+    protected void doInitialize(final Context context) {
+        // Required
+        addOption(CommandOption.POLICY_ID.createOption());
+
+        // Optional
+        addOption(CommandOption.POLICY_RESOURCE.createOption());
+        addOption(CommandOption.POLICY_ACTION.createOption());
+        addOption(CommandOption.USER_NAME_LIST.createOption());
+        addOption(CommandOption.USER_ID_LIST.createOption());
+        addOption(CommandOption.GROUP_NAME_LIST.createOption());
+        addOption(CommandOption.GROUP_ID_LIST.createOption());
+    }
+
+    @Override
+    public StringResult doExecute(final NiFiRegistryClient client, final Properties properties)
+            throws IOException, NiFiRegistryException, ParseException {
+
+        if (!(client instanceof ExtendedNiFiRegistryClient)) {
+            throw new IllegalArgumentException("This command needs extended registry client!");
+        }
+
+        final ExtendedNiFiRegistryClient extendedClient = (ExtendedNiFiRegistryClient) client;
+        final PoliciesClient policiesClient = extendedClient.getPoliciesClient();
+        final TenantsClient tenantsClient = extendedClient.getTenantsClient();
+
+        final String policyId = getRequiredArg(properties, CommandOption.POLICY_ID);
+        final String resource = getArg(properties, CommandOption.POLICY_RESOURCE);
+        final String action = getArg(properties, CommandOption.POLICY_ACTION);
+
+        final AccessPolicy existingPolicy = policiesClient.getPolicy(policyId);
+
+        if (StringUtils.isNotBlank(resource)) {
+            existingPolicy.setResource(resource);
+        }
+
+        if (StringUtils.isNotBlank(action)) {
+            existingPolicy.setAction(action);
+        }
+
+        final Set<Tenant> users = TenantHelper.getExistingUsers(
+                tenantsClient,
+                getArg(properties, CommandOption.USER_NAME_LIST),
+                getArg(properties, CommandOption.USER_ID_LIST));
+
+        if (StringUtils.isNotBlank(getArg(properties, CommandOption.USER_NAME_LIST)) || StringUtils.isNotBlank(getArg(properties, CommandOption.USER_ID_LIST))) {
+            existingPolicy.setUsers(users);
+        }
+
+        final Set<Tenant> userGroups = TenantHelper.getExistingGroups(
+                tenantsClient,
+                getArg(properties, CommandOption.GROUP_NAME_LIST),
+                getArg(properties, CommandOption.GROUP_ID_LIST));
+
+        if (StringUtils.isNotBlank(getArg(properties, CommandOption.GROUP_NAME_LIST)) || StringUtils.isNotBlank(getArg(properties, CommandOption.GROUP_ID_LIST))) {
+            existingPolicy.setUserGroups(userGroups);
+        }
+
+        final AccessPolicy updatedPolicy = policiesClient.updatePolicy(existingPolicy);
+        return new StringResult(updatedPolicy.getIdentifier(), getContext().isInteractive());
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/UpdateUser.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/UpdateUser.java
new file mode 100644
index 0000000..c41f1ba
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/UpdateUser.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.toolkit.cli.impl.command.registry.tenant;
+
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.authorization.User;
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.toolkit.cli.api.Context;
+import org.apache.nifi.toolkit.cli.impl.client.ExtendedNiFiRegistryClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.TenantsClient;
+import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
+import org.apache.nifi.toolkit.cli.impl.command.registry.AbstractNiFiRegistryCommand;
+import org.apache.nifi.toolkit.cli.impl.result.StringResult;
+
+import java.io.IOException;
+import java.util.Properties;
+
+/**
+ * Command for update an existing user.
+ */
+public class UpdateUser extends AbstractNiFiRegistryCommand<StringResult> {
+
+    public UpdateUser() {
+        super("update-user", StringResult.class);
+    }
+
+    @Override
+    public String getDescription() {
+        return "Updates an existing user.";
+    }
+
+    @Override
+    protected void doInitialize(final Context context) {
+        // Required
+        addOption(CommandOption.USER_ID.createOption());
+
+        // Optional
+        addOption(CommandOption.USER_NAME.createOption());
+    }
+
+    @Override
+    public StringResult doExecute(final NiFiRegistryClient client, final Properties properties)
+        throws IOException, NiFiRegistryException, ParseException {
+
+        if (!(client instanceof ExtendedNiFiRegistryClient)) {
+            throw new IllegalArgumentException("This command needs extended registry client!");
+        }
+
+        final ExtendedNiFiRegistryClient extendedClient = (ExtendedNiFiRegistryClient) client;
+        final TenantsClient tenantsClient = extendedClient.getTenantsClient();
+        final String userId = getRequiredArg(properties, CommandOption.USER_ID);
+        final User existingUser = tenantsClient.getUser(userId);
+
+        final String userName = getArg(properties, CommandOption.USER_NAME);
+
+        if (StringUtils.isNotBlank(userName)) {
+            existingUser.setIdentity(userName);
+        }
+
+        final User updatedUser = tenantsClient.updateUser(existingUser);
+        return new StringResult(updatedUser.getIdentifier(), getContext().isInteractive());
+    }
+}
diff --git a/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/UpdateUserGroup.java b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/UpdateUserGroup.java
new file mode 100644
index 0000000..d29a843
--- /dev/null
+++ b/nifi-toolkit/nifi-toolkit-cli/src/main/java/org/apache/nifi/toolkit/cli/impl/command/registry/tenant/UpdateUserGroup.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.toolkit.cli.impl.command.registry.tenant;
+
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.registry.authorization.Tenant;
+import org.apache.nifi.registry.authorization.UserGroup;
+import org.apache.nifi.registry.client.NiFiRegistryClient;
+import org.apache.nifi.registry.client.NiFiRegistryException;
+import org.apache.nifi.toolkit.cli.api.Context;
+import org.apache.nifi.toolkit.cli.impl.client.ExtendedNiFiRegistryClient;
+import org.apache.nifi.toolkit.cli.impl.client.registry.TenantsClient;
+import org.apache.nifi.toolkit.cli.impl.command.CommandOption;
+import org.apache.nifi.toolkit.cli.impl.command.registry.AbstractNiFiRegistryCommand;
+import org.apache.nifi.toolkit.cli.impl.result.StringResult;
+
+import java.io.IOException;
+import java.util.Properties;
+import java.util.Set;
+
+/**
+ * Command for update an existing user group.
+ */
+public class UpdateUserGroup extends AbstractNiFiRegistryCommand<StringResult> {
+
+    public UpdateUserGroup() {
+        super("update-user-group", StringResult.class);
+    }
+
+    @Override
+    public String getDescription() {
+        return "Updates existing user group.";
+    }
+
+    @Override
+    protected void doInitialize(final Context context) {
+        // Required
+        addOption(CommandOption.UG_ID.createOption());
+
+        // Optional
+        addOption(CommandOption.UG_NAME.createOption());
+        addOption(CommandOption.USER_NAME_LIST.createOption());
+        addOption(CommandOption.USER_ID_LIST.createOption());
+    }
+
+    @Override
+    public StringResult doExecute(final NiFiRegistryClient client, final Properties properties)
+            throws IOException, NiFiRegistryException, ParseException {
+        if (!(client instanceof ExtendedNiFiRegistryClient)) {
+            throw new IllegalArgumentException("This command needs extended registry client!");
+        }
+
+        final ExtendedNiFiRegistryClient extendedClient = (ExtendedNiFiRegistryClient) client;
+        final TenantsClient tenantsClient = extendedClient.getTenantsClient();
+        final String groupId = getRequiredArg(properties, CommandOption.UG_ID);
+        final UserGroup existingGroup = tenantsClient.getUserGroup(groupId);
+
+        // Update group name
+        final String groupName = getArg(properties, CommandOption.UG_NAME);
+
+        if (StringUtils.isNotBlank(groupName)) {
+            existingGroup.setIdentity(groupName);
+        }
+
+        // Update group members
+        final Set<Tenant> tenants = TenantHelper.getExistingUsers(
+                tenantsClient,
+                getArg(properties, CommandOption.USER_NAME_LIST),
+                getArg(properties, CommandOption.USER_ID_LIST));
+
+        existingGroup.setUsers(tenants);
+
+        final UserGroup updatedGroup = tenantsClient.updateUserGroup(existingGroup);
+        return new StringResult(updatedGroup.getIdentifier(), getContext().isInteractive());
+    }
+}