You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/10/21 18:53:18 UTC

[GitHub] [ozone] avijayanhwx opened a new pull request #2759: HDDS-5834 [Multi-Tenant] Implement ListUsersInTenant, Remove in memory maps.

avijayanhwx opened a new pull request #2759:
URL: https://github.com/apache/ozone/pull/2759


   ## What changes were proposed in this pull request?
   
   - List Tenant Users API.
   - Remove in memory map in multi tenant manager impl.
   - Code Refactoring
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-5834
   
   ## How was this patch tested?
   Manually tested.
   Unit tests pending.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] prashantpogde commented on a change in pull request #2759: HDDS-5834 [Multi-Tenant] Implement ListUsersInTenant, Remove in memory maps.

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on a change in pull request #2759:
URL: https://github.com/apache/ozone/pull/2759#discussion_r742177763



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
##########
@@ -523,4 +523,31 @@ public void testOzoneTenantBasicOperations() throws IOException {
     checkOutput(err, "Revoked accessId", false);
   }
 
-}
+  private void testListTenantUsers() throws IOException {

Review comment:
       what happens when we don't give tenant argument ? will it list all the kerberos principals ?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMultiTenantManager.java
##########
@@ -44,15 +44,14 @@
    *       . OM-DB state <-in-sync-> IMultiTenantGateKeeperPluginState
    *       . OM DB state is always the source of truth.
    *
-   * @param configuration
    * @throws IOException
    */
-  void start(OzoneConfiguration configuration) throws IOException;
-
-  /**
-   * Stop multi-tenant manager.
-   */
-  void stop() throws Exception;
+//  void start() throws IOException;

Review comment:
       this could have been place to initialize ranger java client plugin too. but we can add this later too. 

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantListUsersHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.hadoop.ozone.shell.tenant;
+
+import java.io.IOException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.om.helpers.TenantUserList;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.TenantUserAccessId;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.s3.S3Handler;
+
+import picocli.CommandLine;
+
+/**
+ * Command to list users in a tenant along with corresponding accessId.
+ */
+@CommandLine.Command(name = "list",
+    description = "List Tenant Users")
+public class TenantListUsersHandler extends S3Handler {
+
+  @CommandLine.Spec
+  private CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.Option(names = {"-t", "--tenant"},
+      description = "Tenant name")
+  private String tenantName;
+
+  @CommandLine.Option(names = {"-p", "--prefix"},
+      description = "Filter users with this prefix.")
+  private String prefix;

Review comment:
       perhaps an option to list only all the admins for a tenant (nice to have p2)

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMultiTenantManagerImpl.java
##########
@@ -427,16 +360,53 @@ public void deactivateUser(String accessID)
   }
 
   @Override
-  public List<String> listAllAccessIDs(String tenantID)
+  public boolean isTenantAdmin(String user, String tenantName) {
+    return true;
+  }
+
+  @Override
+  public TenantUserList listUsersInTenant(String tenantID, String prefix)
       throws IOException {
-    return null;
+
+    if (!omMetadataManager.getTenantStateTable().isExist(tenantID)) {
+      throw new IOException("Tenant '" + tenantID + "' not found!");
+    }
+
+    List<TenantUserAccessId> userAccessIds = new ArrayList<>();
+    CachedTenantInfo cachedTenantInfo = tenantCache.get(tenantID);
+    if (cachedTenantInfo == null) {
+      throw new IOException("Inconsistent in memory Tenant cache '" + tenantID
+          + "' not found in cache, but present in OM DB!");
+    }
+
+    cachedTenantInfo.getTenantUsers().stream()
+        .filter(
+            k -> StringUtils.isEmpty(prefix) || k.getKey().startsWith(prefix))
+        .forEach(
+            k -> userAccessIds.add(
+                TenantUserAccessId.newBuilder()
+                    .setUser(k.getKey())
+                    .setAccessId(k.getValue())
+                    .build()));
+    return new TenantUserList(tenantID, userAccessIds);

Review comment:
       when we list accessIDs for a tenant, should we also indicate that which users are the admin users (* or something )?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] prashantpogde commented on a change in pull request #2759: HDDS-5834 [Multi-Tenant] Implement ListUsersInTenant, Remove in memory maps.

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on a change in pull request #2759:
URL: https://github.com/apache/ozone/pull/2759#discussion_r742177763



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
##########
@@ -523,4 +523,31 @@ public void testOzoneTenantBasicOperations() throws IOException {
     checkOutput(err, "Revoked accessId", false);
   }
 
-}
+  private void testListTenantUsers() throws IOException {

Review comment:
       what happens when we don't give tenant argument ? will it list all the kerberos principals ?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMultiTenantManager.java
##########
@@ -44,15 +44,14 @@
    *       . OM-DB state <-in-sync-> IMultiTenantGateKeeperPluginState
    *       . OM DB state is always the source of truth.
    *
-   * @param configuration
    * @throws IOException
    */
-  void start(OzoneConfiguration configuration) throws IOException;
-
-  /**
-   * Stop multi-tenant manager.
-   */
-  void stop() throws Exception;
+//  void start() throws IOException;

Review comment:
       this could have been place to initialize ranger java client plugin too. but we can add this later too. 

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantListUsersHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.hadoop.ozone.shell.tenant;
+
+import java.io.IOException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.om.helpers.TenantUserList;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.TenantUserAccessId;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.s3.S3Handler;
+
+import picocli.CommandLine;
+
+/**
+ * Command to list users in a tenant along with corresponding accessId.
+ */
+@CommandLine.Command(name = "list",
+    description = "List Tenant Users")
+public class TenantListUsersHandler extends S3Handler {
+
+  @CommandLine.Spec
+  private CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.Option(names = {"-t", "--tenant"},
+      description = "Tenant name")
+  private String tenantName;
+
+  @CommandLine.Option(names = {"-p", "--prefix"},
+      description = "Filter users with this prefix.")
+  private String prefix;

Review comment:
       perhaps an option to list only all the admins for a tenant (nice to have p2)

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMultiTenantManagerImpl.java
##########
@@ -427,16 +360,53 @@ public void deactivateUser(String accessID)
   }
 
   @Override
-  public List<String> listAllAccessIDs(String tenantID)
+  public boolean isTenantAdmin(String user, String tenantName) {
+    return true;
+  }
+
+  @Override
+  public TenantUserList listUsersInTenant(String tenantID, String prefix)
       throws IOException {
-    return null;
+
+    if (!omMetadataManager.getTenantStateTable().isExist(tenantID)) {
+      throw new IOException("Tenant '" + tenantID + "' not found!");
+    }
+
+    List<TenantUserAccessId> userAccessIds = new ArrayList<>();
+    CachedTenantInfo cachedTenantInfo = tenantCache.get(tenantID);
+    if (cachedTenantInfo == null) {
+      throw new IOException("Inconsistent in memory Tenant cache '" + tenantID
+          + "' not found in cache, but present in OM DB!");
+    }
+
+    cachedTenantInfo.getTenantUsers().stream()
+        .filter(
+            k -> StringUtils.isEmpty(prefix) || k.getKey().startsWith(prefix))
+        .forEach(
+            k -> userAccessIds.add(
+                TenantUserAccessId.newBuilder()
+                    .setUser(k.getKey())
+                    .setAccessId(k.getValue())
+                    .build()));
+    return new TenantUserList(tenantID, userAccessIds);

Review comment:
       when we list accessIDs for a tenant, should we also indicate that which users are the admin users (* or something )?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] avijayanhwx commented on a change in pull request #2759: HDDS-5834 [Multi-Tenant] Implement ListUsersInTenant, Remove in memory maps.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #2759:
URL: https://github.com/apache/ozone/pull/2759#discussion_r742989489



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantListUsersHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.hadoop.ozone.shell.tenant;
+
+import java.io.IOException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.om.helpers.TenantUserList;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.TenantUserAccessId;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.s3.S3Handler;
+
+import picocli.CommandLine;
+
+/**
+ * Command to list users in a tenant along with corresponding accessId.
+ */
+@CommandLine.Command(name = "list",
+    description = "List Tenant Users")
+public class TenantListUsersHandler extends S3Handler {
+
+  @CommandLine.Spec
+  private CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.Option(names = {"-t", "--tenant"},
+      description = "Tenant name")
+  private String tenantName;
+
+  @CommandLine.Option(names = {"-p", "--prefix"},
+      description = "Filter users with this prefix.")
+  private String prefix;

Review comment:
       Yes, we can include it in the follow up task as mentioned in the last comment.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] avijayanhwx merged pull request #2759: HDDS-5834 [Multi-Tenant] Implement ListUsersInTenant, Remove in memory maps.

Posted by GitBox <gi...@apache.org>.
avijayanhwx merged pull request #2759:
URL: https://github.com/apache/ozone/pull/2759


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] avijayanhwx commented on a change in pull request #2759: HDDS-5834 [Multi-Tenant] Implement ListUsersInTenant, Remove in memory maps.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #2759:
URL: https://github.com/apache/ozone/pull/2759#discussion_r742987957



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMultiTenantManagerImpl.java
##########
@@ -427,16 +360,53 @@ public void deactivateUser(String accessID)
   }
 
   @Override
-  public List<String> listAllAccessIDs(String tenantID)
+  public boolean isTenantAdmin(String user, String tenantName) {
+    return true;
+  }
+
+  @Override
+  public TenantUserList listUsersInTenant(String tenantID, String prefix)
       throws IOException {
-    return null;
+
+    if (!omMetadataManager.getTenantStateTable().isExist(tenantID)) {
+      throw new IOException("Tenant '" + tenantID + "' not found!");
+    }
+
+    List<TenantUserAccessId> userAccessIds = new ArrayList<>();
+    CachedTenantInfo cachedTenantInfo = tenantCache.get(tenantID);
+    if (cachedTenantInfo == null) {
+      throw new IOException("Inconsistent in memory Tenant cache '" + tenantID
+          + "' not found in cache, but present in OM DB!");
+    }
+
+    cachedTenantInfo.getTenantUsers().stream()
+        .filter(
+            k -> StringUtils.isEmpty(prefix) || k.getKey().startsWith(prefix))
+        .forEach(
+            k -> userAccessIds.add(
+                TenantUserAccessId.newBuilder()
+                    .setUser(k.getKey())
+                    .setAccessId(k.getValue())
+                    .build()));
+    return new TenantUserList(tenantID, userAccessIds);

Review comment:
       Yes, I have created an internal tracking task for that. We can handle it in a follow up task.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] avijayanhwx commented on pull request #2759: HDDS-5834 [Multi-Tenant] Implement ListUsersInTenant, Remove in memory maps.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on pull request #2759:
URL: https://github.com/apache/ozone/pull/2759#issuecomment-961192164


   Thank you for the review @prashantpogde. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] prashantpogde commented on a change in pull request #2759: HDDS-5834 [Multi-Tenant] Implement ListUsersInTenant, Remove in memory maps.

Posted by GitBox <gi...@apache.org>.
prashantpogde commented on a change in pull request #2759:
URL: https://github.com/apache/ozone/pull/2759#discussion_r742177763



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
##########
@@ -523,4 +523,31 @@ public void testOzoneTenantBasicOperations() throws IOException {
     checkOutput(err, "Revoked accessId", false);
   }
 
-}
+  private void testListTenantUsers() throws IOException {

Review comment:
       what happens when we don't give tenant argument ? will it list all the kerberos principals ?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMultiTenantManager.java
##########
@@ -44,15 +44,14 @@
    *       . OM-DB state <-in-sync-> IMultiTenantGateKeeperPluginState
    *       . OM DB state is always the source of truth.
    *
-   * @param configuration
    * @throws IOException
    */
-  void start(OzoneConfiguration configuration) throws IOException;
-
-  /**
-   * Stop multi-tenant manager.
-   */
-  void stop() throws Exception;
+//  void start() throws IOException;

Review comment:
       this could have been place to initialize ranger java client plugin too. but we can add this later too. 

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantListUsersHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.hadoop.ozone.shell.tenant;
+
+import java.io.IOException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.om.helpers.TenantUserList;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.TenantUserAccessId;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.s3.S3Handler;
+
+import picocli.CommandLine;
+
+/**
+ * Command to list users in a tenant along with corresponding accessId.
+ */
+@CommandLine.Command(name = "list",
+    description = "List Tenant Users")
+public class TenantListUsersHandler extends S3Handler {
+
+  @CommandLine.Spec
+  private CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.Option(names = {"-t", "--tenant"},
+      description = "Tenant name")
+  private String tenantName;
+
+  @CommandLine.Option(names = {"-p", "--prefix"},
+      description = "Filter users with this prefix.")
+  private String prefix;

Review comment:
       perhaps an option to list only all the admins for a tenant (nice to have p2)

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMultiTenantManagerImpl.java
##########
@@ -427,16 +360,53 @@ public void deactivateUser(String accessID)
   }
 
   @Override
-  public List<String> listAllAccessIDs(String tenantID)
+  public boolean isTenantAdmin(String user, String tenantName) {
+    return true;
+  }
+
+  @Override
+  public TenantUserList listUsersInTenant(String tenantID, String prefix)
       throws IOException {
-    return null;
+
+    if (!omMetadataManager.getTenantStateTable().isExist(tenantID)) {
+      throw new IOException("Tenant '" + tenantID + "' not found!");
+    }
+
+    List<TenantUserAccessId> userAccessIds = new ArrayList<>();
+    CachedTenantInfo cachedTenantInfo = tenantCache.get(tenantID);
+    if (cachedTenantInfo == null) {
+      throw new IOException("Inconsistent in memory Tenant cache '" + tenantID
+          + "' not found in cache, but present in OM DB!");
+    }
+
+    cachedTenantInfo.getTenantUsers().stream()
+        .filter(
+            k -> StringUtils.isEmpty(prefix) || k.getKey().startsWith(prefix))
+        .forEach(
+            k -> userAccessIds.add(
+                TenantUserAccessId.newBuilder()
+                    .setUser(k.getKey())
+                    .setAccessId(k.getValue())
+                    .build()));
+    return new TenantUserList(tenantID, userAccessIds);

Review comment:
       when we list accessIDs for a tenant, should we also indicate that which users are the admin users (* or something )?

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
##########
@@ -523,4 +523,31 @@ public void testOzoneTenantBasicOperations() throws IOException {
     checkOutput(err, "Revoked accessId", false);
   }
 
-}
+  private void testListTenantUsers() throws IOException {

Review comment:
       what happens when we don't give tenant argument ? will it list all the kerberos principals ?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMultiTenantManager.java
##########
@@ -44,15 +44,14 @@
    *       . OM-DB state <-in-sync-> IMultiTenantGateKeeperPluginState
    *       . OM DB state is always the source of truth.
    *
-   * @param configuration
    * @throws IOException
    */
-  void start(OzoneConfiguration configuration) throws IOException;
-
-  /**
-   * Stop multi-tenant manager.
-   */
-  void stop() throws Exception;
+//  void start() throws IOException;

Review comment:
       this could have been place to initialize ranger java client plugin too. but we can add this later too. 

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantListUsersHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.hadoop.ozone.shell.tenant;
+
+import java.io.IOException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.om.helpers.TenantUserList;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.TenantUserAccessId;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.s3.S3Handler;
+
+import picocli.CommandLine;
+
+/**
+ * Command to list users in a tenant along with corresponding accessId.
+ */
+@CommandLine.Command(name = "list",
+    description = "List Tenant Users")
+public class TenantListUsersHandler extends S3Handler {
+
+  @CommandLine.Spec
+  private CommandLine.Model.CommandSpec spec;
+
+  @CommandLine.Option(names = {"-t", "--tenant"},
+      description = "Tenant name")
+  private String tenantName;
+
+  @CommandLine.Option(names = {"-p", "--prefix"},
+      description = "Filter users with this prefix.")
+  private String prefix;

Review comment:
       perhaps an option to list only all the admins for a tenant (nice to have p2)

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMultiTenantManagerImpl.java
##########
@@ -427,16 +360,53 @@ public void deactivateUser(String accessID)
   }
 
   @Override
-  public List<String> listAllAccessIDs(String tenantID)
+  public boolean isTenantAdmin(String user, String tenantName) {
+    return true;
+  }
+
+  @Override
+  public TenantUserList listUsersInTenant(String tenantID, String prefix)
       throws IOException {
-    return null;
+
+    if (!omMetadataManager.getTenantStateTable().isExist(tenantID)) {
+      throw new IOException("Tenant '" + tenantID + "' not found!");
+    }
+
+    List<TenantUserAccessId> userAccessIds = new ArrayList<>();
+    CachedTenantInfo cachedTenantInfo = tenantCache.get(tenantID);
+    if (cachedTenantInfo == null) {
+      throw new IOException("Inconsistent in memory Tenant cache '" + tenantID
+          + "' not found in cache, but present in OM DB!");
+    }
+
+    cachedTenantInfo.getTenantUsers().stream()
+        .filter(
+            k -> StringUtils.isEmpty(prefix) || k.getKey().startsWith(prefix))
+        .forEach(
+            k -> userAccessIds.add(
+                TenantUserAccessId.newBuilder()
+                    .setUser(k.getKey())
+                    .setAccessId(k.getValue())
+                    .build()));
+    return new TenantUserList(tenantID, userAccessIds);

Review comment:
       when we list accessIDs for a tenant, should we also indicate that which users are the admin users (* or something )?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] avijayanhwx merged pull request #2759: HDDS-5834 [Multi-Tenant] Implement ListUsersInTenant, Remove in memory maps.

Posted by GitBox <gi...@apache.org>.
avijayanhwx merged pull request #2759:
URL: https://github.com/apache/ozone/pull/2759


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] avijayanhwx commented on a change in pull request #2759: HDDS-5834 [Multi-Tenant] Implement ListUsersInTenant, Remove in memory maps.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #2759:
URL: https://github.com/apache/ozone/pull/2759#discussion_r742987542



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMultiTenantManager.java
##########
@@ -44,15 +44,14 @@
    *       . OM-DB state <-in-sync-> IMultiTenantGateKeeperPluginState
    *       . OM DB state is always the source of truth.
    *
-   * @param configuration
    * @throws IOException
    */
-  void start(OzoneConfiguration configuration) throws IOException;
-
-  /**
-   * Stop multi-tenant manager.
-   */
-  void stop() throws Exception;
+//  void start() throws IOException;

Review comment:
       Yes, agreed. That is why did not remove, just commented out.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] avijayanhwx commented on a change in pull request #2759: HDDS-5834 [Multi-Tenant] Implement ListUsersInTenant, Remove in memory maps.

Posted by GitBox <gi...@apache.org>.
avijayanhwx commented on a change in pull request #2759:
URL: https://github.com/apache/ozone/pull/2759#discussion_r742989039



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
##########
@@ -523,4 +523,31 @@ public void testOzoneTenantBasicOperations() throws IOException {
     checkOutput(err, "Revoked accessId", false);
   }
 
-}
+  private void testListTenantUsers() throws IOException {

Review comment:
       The command mandates a "tenant" to be passed in. Else, it will fail.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org