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/11/20 02:57:22 UTC

[GitHub] [ozone] smengcl opened a new pull request #2857: HDDS-6022. [WIP] [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

smengcl opened a new pull request #2857:
URL: https://github.com/apache/ozone/pull/2857


   https://issues.apache.org/jira/browse/HDDS-6022
   
   - [x] Add a new field `refCount` for Ozone volumes that essentially acts like a lock which would prevent the volume from being deleted if `refCount > 0`
   - [ ] Implement `ozone tenant delete`. This command will only remove all tenant information from DB (and Ranger but maybe delayed, depending on the Ranger background thread impl) if there are **zero** users assigned to this tenant and the volume is **empty**.
   
   ## Testing
   
   - [ ] Add new integration test cases.


-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
##########
@@ -84,6 +97,14 @@ private OmVolumeArgs(String adminName, String ownerName, String volume,
     this.updateID = updateID;
   }
 
+  public long getRefCount() {
+    assert (refCount >= 0);

Review comment:
       Ah. I added this `assert` for potential debugging use. Since Java asserts are ignored in production.
   
   I could change this into `Preconditions.checkState`.




-- 
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] smengcl commented on pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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


   > Thanks for working on this @smengcl. Large PR so lots of comments : ) At a high level there are a few things I'd like to mention:
   > 
   > 1. I didn't see much testing of tenant delete beyond just that the command succeeded. Let me know if I missed something, but it looks like we need more in depth testing of the command to check that volume is removed, remaining tenant state is cleared up, etc.
   > 2. I think we should consider the ref count idea a bit more carefully. Right now nothing decrements the count, so if someone else uses it for something and does the same thing we did the volume will be stuck forever. Decrementing the count for us would imply an alternate tenant delete variation which deletes the tenant state but not the volume, decrementing the count. I think this might be a good thing to have but we should discuss.
   
   Thanks @errose28 for the review.
   
   1.
   There is one negative test case in the integration test, that checks the tenant can't be removed if there are accessIds assigned: https://github.com/apache/ozone/pull/2857/files#diff-272c4adc4bf2a4f0e6673d4fff2eb07bf0a8f1614d38e4e7b18cdf2b98b0af5bR563-R568
   
   and robot test: https://github.com/apache/ozone/pull/2857/files#diff-4ee40ef9539d7c861f0deef0d3d9ef8e7a131c2cf44e6f5c7892677f9e8ac56fR63-R65
   
   No tests for volume emptiness / volume existence negative tests have been added yet.
   
   2. Yes. In an earlier unpublished iteration I do decrease the volume refCount, but that logic was later replaced by the thrown exception. Since how `DeleteTenantRequest` should behave when `refCount > 1L` is unclear here. (Should we just keep the volume and only decrease the refCount? Probably no, because the Ranger policies will be removed regardless that renders the volume inaccessible to all users but cluster admins). So the assumption made here is that the volume `refCount` MUST be `<=1` (or == 1 to be exact) to pass this check. -- Essentially all other new features in the future that potentially increment the volume `refCount` must be disable on this volume first.


-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -99,26 +100,29 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantId) {
+      String tenantId) throws OMException {
 
     final OmDBTenantInfo tenantInfo;
     try {
       tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);

Review comment:
       We don't want this one in a try/catch. An exception here does not mean the tenant was not found, it means there was an error accessing RocksDB. This should terminate the OM. OzoneManagerRatisUtils#exceptionToResponseState will read the IOException to determine whether it warrants an error response to the user or OM termination. This means this method should throw IOException. Error handling in the two cases below looks good though.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -99,26 +100,29 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantId) {
+      String tenantId) throws OMException {
 
     final OmDBTenantInfo tenantInfo;
     try {
       tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);

Review comment:
       Makes sense. done




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java
##########
@@ -389,8 +392,8 @@ private void unlock(Resource resource, String resourceName,
     USER_LOCK((byte) 3, "USER_LOCK"), // 15
 
     S3_SECRET_LOCK((byte) 4, "S3_SECRET_LOCK"), // 31
-    PREFIX_LOCK((byte) 5, "PREFIX_LOCK"); //63
-//    TENANT_LOCK((byte) 6, "TENANT_LOCK"); // 127
+    PREFIX_LOCK((byte) 5, "PREFIX_LOCK"), //63
+    TENANT_LOCK((byte) 6, "TENANT_LOCK"); // 127

Review comment:
       Assign TENANT_LOCK weight 0.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantCreateHandler.java
##########
@@ -33,26 +32,19 @@
     description = "Create one or more tenants")
 public class TenantCreateHandler extends TenantHandler {
 
-  @CommandLine.Spec
-  private CommandLine.Model.CommandSpec spec;
-
-  @CommandLine.Parameters(description = "List of tenant names")
+  @CommandLine.Parameters(description = "List of tenant names", arity = "1..")
   private List<String> tenants = new ArrayList<>();
 
   @Override
   protected void execute(OzoneClient client, OzoneAddress address) {
-    if (tenants.size() > 0) {
-      for (String tenantName : tenants) {
-        try {
-          client.getObjectStore().createTenant(tenantName);
-          out().println("Created tenant '" + tenantName + "'.");
-        } catch (IOException e) {
-          err().println("Failed to create tenant '" + tenantName + "': " +
-              e.getMessage());
-        }
+    for (String tenantId : tenants) {

Review comment:
       Removed loop in `CreateTenant` and `DeleteTenant`. Might remove others in another refactoring jira.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -94,26 +99,26 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantName) {
+      String tenantId) {
 
     final OmDBTenantInfo tenantInfo;
     try {
-      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantName);
+      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);
     } catch (IOException e) {
       throw new RuntimeException("Potential DB error. Unable to retrieve "
-          + "OmDBTenantInfo entry for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry for tenant '" + tenantId + "'.");

Review comment:
       done




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,256 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.METADATA_ERROR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IS_REFERENCED;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.TENANT_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {

Review comment:
       Done




-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
##########
@@ -408,7 +410,7 @@ public void testOzoneTenantBasicOperations() throws IOException {
     executeHA(tenantShell, new String[] {"create", "finance"});

Review comment:
       Side note: we should have a jira for the fixme above so we don't forget about it. Or change it to say TODO so we can grep and fix all the TODOs before merging. Unrelated but I just happened to see it.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -94,26 +99,26 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantName) {
+      String tenantId) {
 
     final OmDBTenantInfo tenantInfo;
     try {
-      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantName);
+      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);
     } catch (IOException e) {
       throw new RuntimeException("Potential DB error. Unable to retrieve "
-          + "OmDBTenantInfo entry for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry for tenant '" + tenantId + "'.");

Review comment:
       `OMClientRequest#validateAndUpdateCache` doesn't really propagate any exception.
   
   To throw IOException in this helper method I'd have to add extra try logic in `validateAndUpdateCache` to handle it




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -94,26 +99,26 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantName) {
+      String tenantId) {
 
     final OmDBTenantInfo tenantInfo;
     try {
-      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantName);
+      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);
     } catch (IOException e) {
       throw new RuntimeException("Potential DB error. Unable to retrieve "
-          + "OmDBTenantInfo entry for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry for tenant '" + tenantId + "'.");
     }
 
     if (tenantInfo == null) {
       throw new RuntimeException("Potential DB error or race condition. "
-          + "OmDBTenantInfo entry is missing for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry is missing for tenant '" + tenantId + "'.");
     }
 
-    final String volumeName = tenantInfo.getAccountNamespaceName();
+    final String volumeName = tenantInfo.getBucketNamespaceName();
 
-    if (StringUtils.isEmpty(tenantName)) {
+    if (StringUtils.isEmpty(tenantId)) {

Review comment:
       Good point.
   
   It could also make sense to terminate the OM if the DB is found to be in an undesired state (could be corrupted in some way?) the admin should be notified immediately (i.e. by terminating the OM).
   
   Changed to `VOLUME_NOT_FOUND`.




-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantDeleteHandler.java
##########
@@ -18,18 +18,45 @@
 package org.apache.hadoop.ozone.shell.tenant;
 
 import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantResponse;
 import org.apache.hadoop.ozone.shell.OzoneAddress;
 import picocli.CommandLine;
 
+import java.io.IOException;
+
 /**
  * ozone tenant delete.
  */
-@CommandLine.Command(name = "delete",
-    description = "Delete a tenant")
+@CommandLine.Command(name = "delete", aliases = "remove",
+    description = "Delete an empty tenant. "
+        + "Will not remove the associated volume.")
 public class TenantDeleteHandler extends TenantHandler {
 
+  @CommandLine.Parameters(description = "Tenant name", arity = "1..1")
+  private String tenantId;
+
   @Override
   protected void execute(OzoneClient client, OzoneAddress address) {
-    err().println("Not Implemented.");
+    try {
+      final DeleteTenantResponse resp =
+          client.getObjectStore().deleteTenant(tenantId);
+      out().println("Deleted tenant '" + tenantId + "'.");
+      long volumeRefCount = resp.getVolRefCount();
+      assert(volumeRefCount >= 0L);
+      final String volumeName = resp.getVolumeName();
+      final String extraPrompt =
+          "But the associated volume '" + volumeName + "' is not removed. ";
+      if (volumeRefCount == 0L) {
+        out().println(extraPrompt + "To delete it, run"
+            + "\n    ozone sh volume delete " + volumeName + "\n");
+      } else {
+        out().println(extraPrompt + "And it is still referenced by some other "
+            + "Ozone features (refCount is " + volumeRefCount + ").");
+      }
+    } catch (IOException e) {
+      // Throw exception to make client exit code non-zero
+      throw new RuntimeException("Failed to delete tenant '" + tenantId + "': "

Review comment:
       Same as above, I think the standard is to propagate the IOException here.

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantSetSecretHandler.java
##########
@@ -68,7 +68,7 @@ protected void execute(OzoneClient client, OzoneAddress address)
       if (omEx.getResult().equals(ACCESSID_NOT_FOUND)) {
         // Print to stderr here in order not to contaminate stdout just in
         // case -e is specified.
-        err().println("AccessId '" + accessId + "' doesn't exist");
+        throw new RuntimeException("AccessId '" + accessId + "' doesn't exist");

Review comment:
       Same as above. Propagate the exception.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -94,26 +99,26 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantName) {
+      String tenantId) {
 
     final OmDBTenantInfo tenantInfo;
     try {
-      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantName);
+      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);
     } catch (IOException e) {
       throw new RuntimeException("Potential DB error. Unable to retrieve "
-          + "OmDBTenantInfo entry for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry for tenant '" + tenantId + "'.");

Review comment:
       This IOException should be propagated to the caller (validateAndUpdateCache) since it indicates a problem reading from the DB.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,206 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    return getOmRequest().toBuilder().setUserInfo(getUserInfo()).build();
   }
 
   @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    String volumeName = null;
+    boolean decVolumeRefCount = true;
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs = null;
+
+    try {
+      // Check tenant existence in tenantStateTable
+      if (!omMetadataManager.getTenantStateTable().isExist(tenantId)) {
+        LOG.debug("tenant: {} does not exist", tenantId);
+        throw new OMException("Tenant '" + tenantId + "' does not exist",
+            TENANT_NOT_FOUND);
+      }
+
+      // Reading the TenantStateTable without lock as we don't have or need
+      // a TENANT_LOCK. The assumption is that OmDBTenantInfo is read-only
+      // once it is set during tenant creation.
+      final OmDBTenantInfo dbTenantInfo =
+          omMetadataManager.getTenantStateTable().get(tenantId);
+      volumeName = dbTenantInfo.getBucketNamespaceName();
+      assert(volumeName != null);
+
+      LOG.debug("Tenant '{}' has volume '{}'", tenantId, volumeName);
+      // decVolumeRefCount is true if volumeName is not empty string
+      decVolumeRefCount = volumeName.length() > 0;
+
+      // Acquire the volume lock
+      Preconditions.checkNotNull(volumeName,

Review comment:
       I don't understand this. We are reading from the DB here, and we have full control over what we write to the DB when creating the tenant. So why not just have create tenant always set the volumeName and remove all the extra checks here?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/tenant/OMTenantDeleteResponse.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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.om.response.s3.tenant;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.TENANT_POLICY_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.TENANT_STATE_TABLE;
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.VOLUME_TABLE;
+
+/**
+ * Response for DeleteTenant request.
+ */
+@CleanupTableInfo(cleanupTables = {
+    TENANT_STATE_TABLE,
+    TENANT_POLICY_TABLE,
+    VOLUME_TABLE
+})
+public class OMTenantDeleteResponse extends OMClientResponse {
+
+  private String volumeName;
+  private OmVolumeArgs omVolumeArgs;
+  private String tenantId;
+  private String userPolicyGroupName;
+  private String bucketPolicyGroupName;
+
+  public OMTenantDeleteResponse(@Nonnull OMResponse omResponse,
+                                @Nonnull String volumeName,
+                                @Nullable OmVolumeArgs omVolumeArgs,
+                                @Nonnull String tenantId,
+                                @Nonnull String userPolicyGroupName,
+                                @Nonnull String bucketPolicyGroupName) {
+    super(omResponse);
+    this.volumeName = volumeName;
+    this.omVolumeArgs = omVolumeArgs;
+    this.tenantId = tenantId;
+    this.userPolicyGroupName = userPolicyGroupName;
+    this.bucketPolicyGroupName = bucketPolicyGroupName;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMTenantDeleteResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    omMetadataManager.getTenantStateTable().deleteWithBatch(
+        batchOperation, tenantId);
+
+    omMetadataManager.getTenantPolicyTable().deleteWithBatch(
+        batchOperation, userPolicyGroupName);
+
+    omMetadataManager.getTenantPolicyTable().deleteWithBatch(
+        batchOperation, bucketPolicyGroupName);
+
+    // The rest are the same as OMVolumeDeleteResponse

Review comment:
       Old comment. Might want to change it to clarify that we are updating the volume ref count here.

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantCreateHandler.java
##########
@@ -17,42 +17,33 @@
  */
 package org.apache.hadoop.ozone.shell.tenant;
 
-import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.shell.OzoneAddress;
 import picocli.CommandLine;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 
 /**
  * ozone tenant create.
  */
 @CommandLine.Command(name = "create",
-    description = "Create one or more tenants")
+    description = "Create a tenant."
+        + " This will also create a new Ozone volume for the tenant.")
 public class TenantCreateHandler extends TenantHandler {
 
-  @CommandLine.Spec
-  private CommandLine.Model.CommandSpec spec;
-
-  @CommandLine.Parameters(description = "List of tenant names")
-  private List<String> tenants = new ArrayList<>();
+  @CommandLine.Parameters(description = "Tenant name", arity = "1..1")
+  private String tenantId;
 
   @Override
   protected void execute(OzoneClient client, OzoneAddress address) {
-    if (tenants.size() > 0) {
-      for (String tenantName : tenants) {
-        try {
-          client.getObjectStore().createTenant(tenantName);
-          out().println("Created tenant '" + tenantName + "'.");
-        } catch (IOException e) {
-          err().println("Failed to create tenant '" + tenantName + "': " +
-              e.getMessage());
-        }
-      }
-    } else {
-      GenericCli.missingSubcommand(spec);
+    try {
+      client.getObjectStore().createTenant(tenantId);
+      // TODO: Add return value and print volume name?
+      out().println("Created tenant '" + tenantId + "'.");
+    } catch (IOException e) {

Review comment:
       Other OM CLIs propagate the IOException here, since the parent's execute definition throws it. This gives a full stack trace which is more useful for diagnosing problems.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -94,26 +99,26 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantName) {
+      String tenantId) {
 
     final OmDBTenantInfo tenantInfo;
     try {
-      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantName);
+      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);
     } catch (IOException e) {
       throw new RuntimeException("Potential DB error. Unable to retrieve "
-          + "OmDBTenantInfo entry for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry for tenant '" + tenantId + "'.");
     }
 
     if (tenantInfo == null) {
       throw new RuntimeException("Potential DB error or race condition. "
-          + "OmDBTenantInfo entry is missing for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry is missing for tenant '" + tenantId + "'.");

Review comment:
       This should be thrown as an OMException with result code `TENANT_NOT_FOUND`.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java
##########
@@ -59,6 +59,9 @@
  *   <tr>
  *     <td> 5 </td> <td> Prefix Lock </td>
  *   </tr>
+ *   <tr>
+ *     <td> 6 </td> <td> Tenant Lock </td>
+ *   </tr>

Review comment:
       Minor: update the doc here after removing the tenant lock.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -94,26 +99,26 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantName) {
+      String tenantId) {
 
     final OmDBTenantInfo tenantInfo;
     try {
-      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantName);
+      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);
     } catch (IOException e) {
       throw new RuntimeException("Potential DB error. Unable to retrieve "
-          + "OmDBTenantInfo entry for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry for tenant '" + tenantId + "'.");
     }
 
     if (tenantInfo == null) {
       throw new RuntimeException("Potential DB error or race condition. "
-          + "OmDBTenantInfo entry is missing for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry is missing for tenant '" + tenantId + "'.");
     }
 
-    final String volumeName = tenantInfo.getAccountNamespaceName();
+    final String volumeName = tenantInfo.getBucketNamespaceName();
 
-    if (StringUtils.isEmpty(tenantName)) {
+    if (StringUtils.isEmpty(tenantId)) {

Review comment:
       This should check `volumeName`, not `tenantId`, and should throw an OMException with result code `VOLUME_NOT_FOUND`.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -668,29 +670,63 @@ public void revokeS3Secret(String kerberosID) throws IOException {
    * {@inheritDoc}
    */
   @Override
-  public void createTenant(String tenantName) throws IOException {
-    Preconditions.checkArgument(Strings.isNotBlank(tenantName),
+  public void createTenant(String tenantId) throws IOException {
+    Preconditions.checkArgument(Strings.isNotBlank(tenantId),
         "tenantName cannot be null or empty.");
-    ozoneManagerClient.createTenant(tenantName);
+    ozoneManagerClient.createTenant(tenantId);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void createTenant(String tenantId, OmTenantArgs omTenantArgs)

Review comment:
       Good point.
   
   Actually I only need to pass in `OmTenantArgs` in this case. Note the `OmVolumeArgs` in `createVolume` is a regular Java class (not a protobuf message generated one).




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
##########
@@ -48,6 +48,19 @@
   private long quotaInNamespace;
   private long usedNamespace;
   private List<OzoneAcl> acls;
+  /**
+   * Reference count on this Ozone volume.
+   *
+   * When reference count is larger than zero, it indicates that at least one
+   * "lock" is held on the volume by some Ozone feature (e.g. multi-tenancy).
+   * Volume delete operation will be denied in this case, and user should be
+   * prompted to release the lock first via the interface provided by that
+   * feature.
+   *
+   * Volumes created using CLI, ObjectStore API or upgraded from older OM DB
+   * will have reference count set to zero by default.
+   */
+  private long refCount;

Review comment:
       Good catch.
   
   tho btw the current broken impl still works, refCount is set directly through `omVolumeArgs.setRefCount(1L)` and persisted to the `VolumeTable`. `ozone tenant create vol1` first, then `ozone sh volume delete vol1` should throw. Such issue could be caught in client-server communication, which we have not implemented so far.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-tenant.robot
##########
@@ -72,3 +76,12 @@ Secure Tenant Create Tenant Failure with Regular (non-admin) user
 Secure Tenant SetSecret Failure with Regular (non-admin) user
     ${rc}  ${output} =  Run And Return Rc And Output  ozone tenant user set-secret 'tenantone$bob' --secret=somesecret2 --export
                         Should contain   ${output}         Permission denied. Requested accessId
+
+Secure Tenant Revoke User AccessId Success
+    Run Keyword   Kinit test user     testuser     testuser.keytab
+    ${output} =         Execute          ozone tenant user revoke 'tenantone$bob'
+                        Should contain   ${output}         Revoked accessId 'tenantone$bob'.

Review comment:
       Added




-- 
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] smengcl edited a comment on pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #2857:
URL: https://github.com/apache/ozone/pull/2857#issuecomment-977343073


   > Thanks for working on this @smengcl. Large PR so lots of comments : ) At a high level there are a few things I'd like to mention:
   > 
   > 1. I didn't see much testing of tenant delete beyond just that the command succeeded. Let me know if I missed something, but it looks like we need more in depth testing of the command to check that volume is removed, remaining tenant state is cleared up, etc.
   > 2. I think we should consider the ref count idea a bit more carefully. Right now nothing decrements the count, so if someone else uses it for something and does the same thing we did the volume will be stuck forever. Decrementing the count for us would imply an alternate tenant delete variation which deletes the tenant state but not the volume, decrementing the count. I think this might be a good thing to have but we should discuss.
   
   Thanks @errose28 for the review.
   
   1. There is only one negative test case in the integration test, that checks the tenant can't be removed if there are accessIds assigned: https://github.com/apache/ozone/blob/3d689c924ae27d855f818be9c3f90a000674e81f/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java#L563-L568
   
   And one for acceptance test: https://github.com/apache/ozone/blob/3d689c924ae27d855f818be9c3f90a000674e81f/hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-tenant.robot#L63-L65
   
   No tests for volume emptiness / volume existence negative tests have been added yet.
   
   2. Yes. In an earlier unpublished iteration I do decrease the volume refCount, but that logic was later replaced by the thrown exception. Since how `DeleteTenantRequest` should behave when `refCount > 1L` is unclear here. (Should we just keep the volume and only decrease the refCount? Probably no, because the Ranger policies will be removed regardless that renders the volume inaccessible to all users but cluster admins). So the assumption made here is that the volume `refCount` MUST be `<=1` (or == 1 to be exact) to pass this check. -- Essentially all other new features in the future that potentially increment the volume `refCount` must be disable on this volume first.


-- 
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] smengcl edited a comment on pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #2857:
URL: https://github.com/apache/ozone/pull/2857#issuecomment-977343073


   > Thanks for working on this @smengcl. Large PR so lots of comments : ) At a high level there are a few things I'd like to mention:
   > 
   > 1. I didn't see much testing of tenant delete beyond just that the command succeeded. Let me know if I missed something, but it looks like we need more in depth testing of the command to check that volume is removed, remaining tenant state is cleared up, etc.
   > 2. I think we should consider the ref count idea a bit more carefully. Right now nothing decrements the count, so if someone else uses it for something and does the same thing we did the volume will be stuck forever. Decrementing the count for us would imply an alternate tenant delete variation which deletes the tenant state but not the volume, decrementing the count. I think this might be a good thing to have but we should discuss.
   
   Thanks @errose28 for the review.
   
   1. There is only one negative test case in the integration test, that checks the tenant can't be removed if there are accessIds assigned: https://github.com/apache/ozone/blob/3d689c924ae27d855f818be9c3f90a000674e81f/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java#L563-L568
   
   And one for acceptance test: https://github.com/apache/ozone/blob/3d689c924ae27d855f818be9c3f90a000674e81f/hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-tenant.robot#L63-L65
   
   No tests for volume emptiness / volume existence negative tests have been added yet.
   
   2. Yes. In an earlier unpublished iteration I do decrease the volume refCount, but that logic was later replaced by the thrown exception. Since how `DeleteTenantRequest` should behave when `refCount > 1L` is unclear here. (Should we just keep the volume and only decrease the refCount? Probably no, because the Ranger policies will be removed regardless that renders the volume inaccessible to all users but cluster admins). So the assumption made here is that the volume `refCount` MUST be `<=1` (or == 1 to be exact) to pass this check. -- Essentially all other new features in the future that potentially increment the volume `refCount` must be disable on this volume first, only then the cluster admin can do `tenant delete`. We can discuss it next time.


-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
##########
@@ -444,6 +471,14 @@ public void deleteBucket(String bucketName) throws IOException {
     proxy.deleteBucket(name, bucketName);
   }
 
+  public long getRefCount() {

Review comment:
       Initially I added this because findbugs was complaining that refCount is unused.
   
   I believe we could expose also refCount in `ozone volume info` / `ozone volume update`

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
##########
@@ -444,6 +471,14 @@ public void deleteBucket(String bucketName) throws IOException {
     proxy.deleteBucket(name, bucketName);
   }
 
+  public long getRefCount() {

Review comment:
       Initially I added this because findbugs was complaining that refCount is unused.
   
   I believe we could expose refCount in `ozone volume info` / `ozone volume update` as well.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/VolumeArgs.java
##########
@@ -115,6 +117,10 @@ public long getQuotaInNamespace() {
     return new VolumeArgs.Builder();
   }
 
+  public long getRefCount() {

Review comment:
       Removed `refCount` altogether from `VolumeArgs`.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java
##########
@@ -100,6 +100,14 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
 
       OmVolumeArgs omVolumeArgs = getVolumeInfo(omMetadataManager, volume);
 
+      // Check reference count
+      if (omVolumeArgs.getRefCount() != 0L) {
+        LOG.debug("volume: {} has non-zero ref count. won't delete", volume);
+        throw new OMException("Volume is being used. Use " +
+            "`ozone tenant delete` CLI to delete the volume instead.",
+            OMException.ResultCodes.VOLUME_IN_USE);

Review comment:
       Maybe rename `VOLUME_IN_USE`.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
##########
@@ -48,6 +48,19 @@
   private long quotaInNamespace;
   private long usedNamespace;
   private List<OzoneAcl> acls;
+  /**
+   * Reference count on this Ozone volume.
+   *
+   * When reference count is larger than zero, it indicates that at least one
+   * "lock" is held on the volume by some Ozone feature (e.g. multi-tenancy).
+   * Volume delete operation will be denied in this case, and user should be
+   * prompted to release the lock first via the interface provided by that
+   * feature.
+   *
+   * Volumes created using CLI, ObjectStore API or upgraded from older OM DB
+   * will have reference count set to zero by default.
+   */
+  private long refCount;

Review comment:
       Good catch.
   
   tho btw the current broken impl still works, refCount is set directly through `omVolumeArgs.setRefCount(1L)` and persisted to the `VolumeTable`.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,256 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.METADATA_ERROR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IS_REFERENCED;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.TENANT_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    // Check if there are any accessIds in the tenant
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getBucketNamespaceName();
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be removed?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false;
+    boolean acquiredUserLock = false;
+    boolean acquiredTenantLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string

Review comment:
       Yep. `volumeName` could still be an empty string (zero-length string) if somehow the tenant doesn't have a volume. But it should not be `null`.




-- 
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] smengcl commented on pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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


   Thanks @errose28 for reviewing this huge patch! (didn't expect this be >1KLOC when I started this :D ).


-- 
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] smengcl edited a comment on pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #2857:
URL: https://github.com/apache/ozone/pull/2857#issuecomment-977343073


   > Thanks for working on this @smengcl. Large PR so lots of comments : ) At a high level there are a few things I'd like to mention:
   > 
   > 1. I didn't see much testing of tenant delete beyond just that the command succeeded. Let me know if I missed something, but it looks like we need more in depth testing of the command to check that volume is removed, remaining tenant state is cleared up, etc.
   > 2. I think we should consider the ref count idea a bit more carefully. Right now nothing decrements the count, so if someone else uses it for something and does the same thing we did the volume will be stuck forever. Decrementing the count for us would imply an alternate tenant delete variation which deletes the tenant state but not the volume, decrementing the count. I think this might be a good thing to have but we should discuss.
   
   Thanks @errose28 for the review.
   
   1. There is one negative test case in the integration test, that checks the tenant can't be removed if there are accessIds assigned: https://github.com/apache/ozone/pull/2857/files#diff-272c4adc4bf2a4f0e6673d4fff2eb07bf0a8f1614d38e4e7b18cdf2b98b0af5bR563-R568
   
   And robot test: https://github.com/apache/ozone/pull/2857/files#diff-4ee40ef9539d7c861f0deef0d3d9ef8e7a131c2cf44e6f5c7892677f9e8ac56fR63-R65
   
   No tests for volume emptiness / volume existence negative tests have been added yet.
   
   2. Yes. In an earlier unpublished iteration I do decrease the volume refCount, but that logic was later replaced by the thrown exception. Since how `DeleteTenantRequest` should behave when `refCount > 1L` is unclear here. (Should we just keep the volume and only decrease the refCount? Probably no, because the Ranger policies will be removed regardless that renders the volume inaccessible to all users but cluster admins). So the assumption made here is that the volume `refCount` MUST be `<=1` (or == 1 to be exact) to pass this check. -- Essentially all other new features in the future that potentially increment the volume `refCount` must be disable on this volume first.


-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-tenant.robot
##########
@@ -72,3 +76,12 @@ Secure Tenant Create Tenant Failure with Regular (non-admin) user
 Secure Tenant SetSecret Failure with Regular (non-admin) user
     ${rc}  ${output} =  Run And Return Rc And Output  ozone tenant user set-secret 'tenantone$bob' --secret=somesecret2 --export
                         Should contain   ${output}         Permission denied. Requested accessId
+
+Secure Tenant Revoke User AccessId Success
+    Run Keyword   Kinit test user     testuser     testuser.keytab
+    ${output} =         Execute          ozone tenant user revoke 'tenantone$bob'
+                        Should contain   ${output}         Revoked accessId 'tenantone$bob'.

Review comment:
       We should probably have a negative test case here too, since a bug in this would be bad. Try to use Bob's access ID and make sure it no longer works.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // TODO: Check tenantId validity? Maybe not
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    // Check if the tenant volume is empty (when the volume exists)
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getAssociatedVolumeName();
+    // Note: Looks like isVolumeEmpty() returns true if volume doesn't exist
+    if (volumeName.length() > 0 && !metadataManager.isVolumeEmpty(volumeName)) {
+      LOG.debug("volume: {} is not empty", volumeName);
+      throw new OMException("Tenant volume '" + volumeName + "' is not empty." +
+          " Volume must be emptied before the tenant can be deleted.",
+          VOLUME_NOT_EMPTY);
+    }
+
+    // Check if there are any accessIds in the tenant
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be ignored?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  public void handleRequestFailure(OzoneManager ozoneManager) {
+    super.handleRequestFailure(ozoneManager);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false, acquiredUserLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string
+    final String volumeName =
+        getOmRequest().getDeleteVolumeRequest().getVolumeName();
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs;
+    String volumeOwner = null;
+    // deleteVolume is true if volumeName is not empty string
+    boolean deleteVolume = volumeName.length() > 0;
+    OzoneManagerStorageProtos.PersistedUserVolumeInfo newVolumeList = null;
+
+    try {
+      if (deleteVolume) {
+        // check Acl
+        if (ozoneManager.getAclsEnabled()) {
+          checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+              OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+              volumeName, null, null);
+        }
+
+        acquiredVolumeLock = omMetadataManager.getLock().acquireWriteLock(
+            VOLUME_LOCK, volumeName);
+
+        omVolumeArgs = getVolumeInfo(omMetadataManager, volumeName);
+        // Check volume ref count
+        long volRefCount = omVolumeArgs.getRefCount();
+        if (volRefCount > 1L) {
+          LOG.warn("Volume '{}' has a greater than 1 reference count of " +
+                  "'{}'", volumeName, volRefCount);
+          throw new OMException("Volume '" + volumeName + "' has a greater " +
+              "than 1 reference count of '" + volRefCount + "'. This may " +
+              "indicate the volume is in use by some other Ozone features. " +
+              "Please disable such other features before trying to delete " +
+              "the tenant again.", VOLUME_IN_USE);
+        }
+
+        volumeOwner = omVolumeArgs.getOwnerName();
+        acquiredUserLock = omMetadataManager.getLock().acquireWriteLock(
+            USER_LOCK, volumeOwner);
+
+        // Check volume emptiness, again
+        if (!omMetadataManager.isVolumeEmpty(volumeName)) {
+          LOG.debug("volume: '{}' is not empty", volumeName);
+          throw new OMException("Aborting tenant deletion. " +
+              "Volume becomes non-empty somewhere between" +
+              "preExecute and validateAndUpdateCache", VOLUME_NOT_EMPTY);
+        }
+
+        // Actual volume deletion, follows OMVolumeDeleteRequest
+        newVolumeList =
+            omMetadataManager.getUserTable().get(volumeOwner);
+        newVolumeList = delVolumeFromOwnerList(newVolumeList, volumeName,
+            volumeOwner, transactionLogIndex);
+        final String dbUserKey = omMetadataManager.getUserKey(volumeOwner);
+        omMetadataManager.getUserTable().addCacheEntry(
+            new CacheKey<>(dbUserKey),
+            new CacheValue<>(Optional.of(newVolumeList), transactionLogIndex));
+        final String dbVolumeKey = omMetadataManager.getVolumeKey(volumeName);
+        omMetadataManager.getVolumeTable().addCacheEntry(
+            new CacheKey<>(dbVolumeKey),
+            new CacheValue<>(Optional.absent(), transactionLogIndex));
+
+        // TODO: Set response dbVolumeKey?
+      }
+
+      // TODO: Should hold some tenant lock here. Just in case !deleteVolume
+//      acquiredTenantLock = omMetadataManager.getLock().acquireWriteLock(
+//          TENANT_LOCK, tenantId);
+
+      // Double check tenant emptiness
+
+      // Invalidate cache entries for tenant
+      omMetadataManager.getTenantStateTable().addCacheEntry(
+          new CacheKey<>(tenantId),
+          new CacheValue<>(Optional.absent(), transactionLogIndex));
+      final String userPolicyGroupName =
+          tenantId + OzoneConsts.DEFAULT_TENANT_USER_POLICY_SUFFIX;
+      final String bucketPolicyGroupName =
+          tenantId + OzoneConsts.DEFAULT_TENANT_BUCKET_POLICY_SUFFIX;

Review comment:
       This will be replaced with `request.getUserPolicyGroupName()` later. The client will be responsible for generating the name by the time.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,256 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.METADATA_ERROR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IS_REFERENCED;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.TENANT_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    // Check if there are any accessIds in the tenant
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {

Review comment:
       You are right.
   I thought I had tenant emptiness check in `validateAndUpdateCache` as well, but apparently not, somehow.
   Done.




-- 
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] smengcl edited a comment on pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #2857:
URL: https://github.com/apache/ozone/pull/2857#issuecomment-984317568


   Known flaky (and unrelated) test `TestHDDSUpgrade` (HDDS-5971). Retriggered CI.


-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,206 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    return getOmRequest().toBuilder().setUserInfo(getUserInfo()).build();
   }
 
   @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    String volumeName = null;
+    boolean decVolumeRefCount = true;
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs = null;
+
+    try {
+      // Check tenant existence in tenantStateTable
+      if (!omMetadataManager.getTenantStateTable().isExist(tenantId)) {
+        LOG.debug("tenant: {} does not exist", tenantId);
+        throw new OMException("Tenant '" + tenantId + "' does not exist",
+            TENANT_NOT_FOUND);
+      }
+
+      // Reading the TenantStateTable without lock as we don't have or need
+      // a TENANT_LOCK. The assumption is that OmDBTenantInfo is read-only
+      // once it is set during tenant creation.
+      final OmDBTenantInfo dbTenantInfo =
+          omMetadataManager.getTenantStateTable().get(tenantId);
+      volumeName = dbTenantInfo.getBucketNamespaceName();
+      assert(volumeName != null);
+
+      LOG.debug("Tenant '{}' has volume '{}'", tenantId, volumeName);
+      // decVolumeRefCount is true if volumeName is not empty string
+      decVolumeRefCount = volumeName.length() > 0;
+
+      // Acquire the volume lock
+      Preconditions.checkNotNull(volumeName,

Review comment:
       This really is an assertion. So that in some other potential buggy write path we could catch it asap (instead of propagating it further). I guess now that we won't have UpdateTenant, CreateTenant is indeed the only write path, for now. Unless the DB is tampered with we shouldn't encounter `volumeName == null` at all.
   
   Also I just realized I already had a `assert(volumeName != null)` earlier. So I'm just going to remove this line. (though assert keyword doesn't trigger in prod)




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,206 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    return getOmRequest().toBuilder().setUserInfo(getUserInfo()).build();
   }
 
   @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    String volumeName = null;
+    boolean decVolumeRefCount = true;
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs = null;
+
+    try {
+      // Check tenant existence in tenantStateTable
+      if (!omMetadataManager.getTenantStateTable().isExist(tenantId)) {
+        LOG.debug("tenant: {} does not exist", tenantId);
+        throw new OMException("Tenant '" + tenantId + "' does not exist",
+            TENANT_NOT_FOUND);
+      }
+
+      // Reading the TenantStateTable without lock as we don't have or need
+      // a TENANT_LOCK. The assumption is that OmDBTenantInfo is read-only
+      // once it is set during tenant creation.
+      final OmDBTenantInfo dbTenantInfo =
+          omMetadataManager.getTenantStateTable().get(tenantId);
+      volumeName = dbTenantInfo.getBucketNamespaceName();
+      assert(volumeName != null);
+
+      LOG.debug("Tenant '{}' has volume '{}'", tenantId, volumeName);
+      // decVolumeRefCount is true if volumeName is not empty string
+      decVolumeRefCount = volumeName.length() > 0;
+
+      // Acquire the volume lock
+      Preconditions.checkNotNull(volumeName,

Review comment:
       This really is an assertion. So that in some other potential buggy write path we could catch it asap (instead of propagating it further). I guess now that we won't have UpdateTenant, CreateTenant is indeed the only write path, for now. Unless the DB is tampered with, we shouldn't encounter `volumeName == null` at all.
   
   Also I just realized I already had a `assert(volumeName != null)` earlier. So I'm just going to remove this line. (though assert keyword doesn't trigger in prod)




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantCreateHandler.java
##########
@@ -17,42 +17,33 @@
  */
 package org.apache.hadoop.ozone.shell.tenant;
 
-import org.apache.hadoop.hdds.cli.GenericCli;
 import org.apache.hadoop.ozone.client.OzoneClient;
 import org.apache.hadoop.ozone.shell.OzoneAddress;
 import picocli.CommandLine;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 
 /**
  * ozone tenant create.
  */
 @CommandLine.Command(name = "create",
-    description = "Create one or more tenants")
+    description = "Create a tenant."
+        + " This will also create a new Ozone volume for the tenant.")
 public class TenantCreateHandler extends TenantHandler {
 
-  @CommandLine.Spec
-  private CommandLine.Model.CommandSpec spec;
-
-  @CommandLine.Parameters(description = "List of tenant names")
-  private List<String> tenants = new ArrayList<>();
+  @CommandLine.Parameters(description = "Tenant name", arity = "1..1")
+  private String tenantId;
 
   @Override
   protected void execute(OzoneClient client, OzoneAddress address) {
-    if (tenants.size() > 0) {
-      for (String tenantName : tenants) {
-        try {
-          client.getObjectStore().createTenant(tenantName);
-          out().println("Created tenant '" + tenantName + "'.");
-        } catch (IOException e) {
-          err().println("Failed to create tenant '" + tenantName + "': " +
-              e.getMessage());
-        }
-      }
-    } else {
-      GenericCli.missingSubcommand(spec);
+    try {
+      client.getObjectStore().createTenant(tenantId);
+      // TODO: Add return value and print volume name?
+      out().println("Created tenant '" + tenantId + "'.");
+    } catch (IOException e) {

Review comment:
       done




-- 
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] smengcl edited a comment on pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #2857:
URL: https://github.com/apache/ozone/pull/2857#issuecomment-977343073


   > Thanks for working on this @smengcl. Large PR so lots of comments : ) At a high level there are a few things I'd like to mention:
   > 
   > 1. I didn't see much testing of tenant delete beyond just that the command succeeded. Let me know if I missed something, but it looks like we need more in depth testing of the command to check that volume is removed, remaining tenant state is cleared up, etc.
   > 2. I think we should consider the ref count idea a bit more carefully. Right now nothing decrements the count, so if someone else uses it for something and does the same thing we did the volume will be stuck forever. Decrementing the count for us would imply an alternate tenant delete variation which deletes the tenant state but not the volume, decrementing the count. I think this might be a good thing to have but we should discuss.
   
   Thanks @errose28 for the review.
   
   1. There is only one negative test case in the integration test, that checks the tenant can't be removed if there are accessIds assigned: https://github.com/apache/ozone/blob/3d689c924ae27d855f818be9c3f90a000674e81f/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java#L563-L568
   
   And one for acceptance test: https://github.com/apache/ozone/blob/3d689c924ae27d855f818be9c3f90a000674e81f/hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-tenant.robot#L63-L65
   
   No tests for volume emptiness / volume existence negative tests have been added yet.
   
   2. Yes. In an earlier unpublished iteration I do decrease the volume refCount, but that logic was later replaced by the thrown exception. Since how `DeleteTenantRequest` should behave when `refCount > 1L` is unclear here. (Should we just keep the volume and only decrease the refCount? Probably no, because the Ranger policies will be removed regardless that renders the volume inaccessible to all users but cluster admins). So the assumption made here is that the volume `refCount` MUST be `<=1` (or == 1 to be exact) to pass this check. -- Essentially all other new features in the future that potentially increment the volume `refCount` must be disable on this volume first. We can discuss it next time.


-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // TODO: Check tenantId validity? Maybe not
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    // Check if the tenant volume is empty (when the volume exists)
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getAssociatedVolumeName();
+    // Note: Looks like isVolumeEmpty() returns true if volume doesn't exist
+    if (volumeName.length() > 0 && !metadataManager.isVolumeEmpty(volumeName)) {
+      LOG.debug("volume: {} is not empty", volumeName);
+      throw new OMException("Tenant volume '" + volumeName + "' is not empty." +
+          " Volume must be emptied before the tenant can be deleted.",
+          VOLUME_NOT_EMPTY);
+    }
+
+    // Check if there are any accessIds in the tenant
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be ignored?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  public void handleRequestFailure(OzoneManager ozoneManager) {
+    super.handleRequestFailure(ozoneManager);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false, acquiredUserLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string
+    final String volumeName =
+        getOmRequest().getDeleteVolumeRequest().getVolumeName();
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs;
+    String volumeOwner = null;
+    // deleteVolume is true if volumeName is not empty string
+    boolean deleteVolume = volumeName.length() > 0;
+    OzoneManagerStorageProtos.PersistedUserVolumeInfo newVolumeList = null;
+
+    try {
+      if (deleteVolume) {
+        // check Acl
+        if (ozoneManager.getAclsEnabled()) {
+          checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+              OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+              volumeName, null, null);
+        }
+
+        acquiredVolumeLock = omMetadataManager.getLock().acquireWriteLock(
+            VOLUME_LOCK, volumeName);
+
+        omVolumeArgs = getVolumeInfo(omMetadataManager, volumeName);
+        // Check volume ref count
+        long volRefCount = omVolumeArgs.getRefCount();
+        if (volRefCount > 1L) {

Review comment:
       The check could be `volRefCount != 1L`




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
##########
@@ -48,6 +48,19 @@
   private long quotaInNamespace;
   private long usedNamespace;
   private List<OzoneAcl> acls;
+  /**
+   * Reference count on this Ozone volume.
+   *
+   * When reference count is larger than zero, it indicates that at least one
+   * "lock" is held on the volume by some Ozone feature (e.g. multi-tenancy).
+   * Volume delete operation will be denied in this case, and user should be
+   * prompted to release the lock first via the interface provided by that
+   * feature.
+   *
+   * Volumes created using CLI, ObjectStore API or upgraded from older OM DB
+   * will have reference count set to zero by default.
+   */
+  private long refCount;

Review comment:
       Good catch.
   
   tho btw the current broken impl still works, refCount is set directly through `omVolumeArgs.setRefCount(1L)` and persisted to the `VolumeTable` in `OMTenantCreateRequest#validateAndUpdateCache`. `ozone tenant create vol1` first, then `ozone sh volume delete vol1` should throw. Such issue could be caught in client-server communication, which we have not implemented so far.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/VolumeArgs.java
##########
@@ -115,6 +117,10 @@ public long getQuotaInNamespace() {
     return new VolumeArgs.Builder();
   }
 
+  public long getRefCount() {

Review comment:
       Removed `refCount` field from `VolumeArgs`.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // TODO: Check tenantId validity? Maybe not
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    // Check if the tenant volume is empty (when the volume exists)
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getAssociatedVolumeName();
+    // Note: Looks like isVolumeEmpty() returns true if volume doesn't exist
+    if (volumeName.length() > 0 && !metadataManager.isVolumeEmpty(volumeName)) {
+      LOG.debug("volume: {} is not empty", volumeName);
+      throw new OMException("Tenant volume '" + volumeName + "' is not empty." +
+          " Volume must be emptied before the tenant can be deleted.",
+          VOLUME_NOT_EMPTY);
+    }
+
+    // Check if there are any accessIds in the tenant
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be ignored?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  public void handleRequestFailure(OzoneManager ozoneManager) {
+    super.handleRequestFailure(ozoneManager);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false, acquiredUserLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string
+    final String volumeName =
+        getOmRequest().getDeleteVolumeRequest().getVolumeName();
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs;
+    String volumeOwner = null;
+    // deleteVolume is true if volumeName is not empty string
+    boolean deleteVolume = volumeName.length() > 0;
+    OzoneManagerStorageProtos.PersistedUserVolumeInfo newVolumeList = null;
+
+    try {
+      if (deleteVolume) {
+        // check Acl
+        if (ozoneManager.getAclsEnabled()) {
+          checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+              OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+              volumeName, null, null);
+        }
+
+        acquiredVolumeLock = omMetadataManager.getLock().acquireWriteLock(
+            VOLUME_LOCK, volumeName);
+
+        omVolumeArgs = getVolumeInfo(omMetadataManager, volumeName);
+        // Check volume ref count
+        long volRefCount = omVolumeArgs.getRefCount();
+        if (volRefCount > 1L) {

Review comment:
       Throwing `METADATA_ERROR` if `volRefCount < 1L`




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -668,29 +670,63 @@ public void revokeS3Secret(String kerberosID) throws IOException {
    * {@inheritDoc}
    */
   @Override
-  public void createTenant(String tenantName) throws IOException {
-    Preconditions.checkArgument(Strings.isNotBlank(tenantName),
+  public void createTenant(String tenantId) throws IOException {
+    Preconditions.checkArgument(Strings.isNotBlank(tenantId),
         "tenantName cannot be null or empty.");
-    ozoneManagerClient.createTenant(tenantName);
+    ozoneManagerClient.createTenant(tenantId);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void createTenant(String tenantId, OmTenantArgs omTenantArgs)

Review comment:
       Done.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+

Review comment:
       Yeah you are right. I will just move all the checks that could be delayed into `validateAndUpdateCache`. Checking in preExecute to save a few ms in case of failure isn't worth it.




-- 
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] smengcl edited a comment on pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #2857:
URL: https://github.com/apache/ozone/pull/2857#issuecomment-977343073


   > Thanks for working on this @smengcl. Large PR so lots of comments : ) At a high level there are a few things I'd like to mention:
   > 
   > 1. I didn't see much testing of tenant delete beyond just that the command succeeded. Let me know if I missed something, but it looks like we need more in depth testing of the command to check that volume is removed, remaining tenant state is cleared up, etc.
   > 2. I think we should consider the ref count idea a bit more carefully. Right now nothing decrements the count, so if someone else uses it for something and does the same thing we did the volume will be stuck forever. Decrementing the count for us would imply an alternate tenant delete variation which deletes the tenant state but not the volume, decrementing the count. I think this might be a good thing to have but we should discuss.
   
   Thanks @errose28 for the review.
   
   1. There is only one negative test case in the integration test, that checks the tenant can't be removed if there are accessIds assigned: https://github.com/apache/ozone/blob/3d689c924ae27d855f818be9c3f90a000674e81f/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java#L563-L568
   
   And robot test: https://github.com/apache/ozone/pull/2857/files#diff-4ee40ef9539d7c861f0deef0d3d9ef8e7a131c2cf44e6f5c7892677f9e8ac56fR63-R65
   
   No tests for volume emptiness / volume existence negative tests have been added yet.
   
   2. Yes. In an earlier unpublished iteration I do decrease the volume refCount, but that logic was later replaced by the thrown exception. Since how `DeleteTenantRequest` should behave when `refCount > 1L` is unclear here. (Should we just keep the volume and only decrease the refCount? Probably no, because the Ranger policies will be removed regardless that renders the volume inaccessible to all users but cluster admins). So the assumption made here is that the volume `refCount` MUST be `<=1` (or == 1 to be exact) to pass this check. -- Essentially all other new features in the future that potentially increment the volume `refCount` must be disable on this volume first.


-- 
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] smengcl edited a comment on pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #2857:
URL: https://github.com/apache/ozone/pull/2857#issuecomment-977343073


   > Thanks for working on this @smengcl. Large PR so lots of comments : ) At a high level there are a few things I'd like to mention:
   > 
   > 1. I didn't see much testing of tenant delete beyond just that the command succeeded. Let me know if I missed something, but it looks like we need more in depth testing of the command to check that volume is removed, remaining tenant state is cleared up, etc.
   > 2. I think we should consider the ref count idea a bit more carefully. Right now nothing decrements the count, so if someone else uses it for something and does the same thing we did the volume will be stuck forever. Decrementing the count for us would imply an alternate tenant delete variation which deletes the tenant state but not the volume, decrementing the count. I think this might be a good thing to have but we should discuss.
   
   Thanks @errose28 for the review.
   
   1. There is only one negative test case in the integration test, that checks the tenant can't be removed if there are accessIds assigned: https://github.com/apache/ozone/blob/3d689c924ae27d855f818be9c3f90a000674e81f/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java#L563-L568
   
   And one for acceptance test: https://github.com/apache/ozone/blob/3d689c924ae27d855f818be9c3f90a000674e81f/hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-tenant.robot#L63-L65
   
   No tests for volume emptiness / volume existence negative tests have been added yet.
   
   2. Yes. In an earlier unpublished iteration I do decrease the volume refCount, but that logic was later replaced by the thrown exception. Since how `DeleteTenantRequest` should behave when `refCount > 1L` is unclear here. (Should we just keep the volume and only decrease the refCount? Probably no, because the Ranger policies will be removed regardless, which renders the volume inaccessible to all users but cluster admins). So the assumption made here is that the volume `refCount` MUST be `<=1` (or == 1 to be exact) to pass this check. -- Essentially all other new features in the future that potentially increment the volume `refCount` must be disable on this volume first, only then the cluster admin can do `tenant delete`. We can discuss it next time.


-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmTenantArgs.java
##########
@@ -22,14 +22,26 @@
  * This class is used for storing Ozone tenant arguments.
  */
 public class OmTenantArgs {
-  /* Tenant name */
+  /**
+   * Tenant name.
+   */
   private final String tenantId;
+  /**
+   * Volume name to be created for this tenant.
+   * Default volume name would be the same as tenant name if unspecified.

Review comment:
       You mean adding a new constructor like this one?:
   
   ```java
     public OmTenantArgs(String tenantId) {
       this.tenantId = tenantId;
       this.volumeName = tenantId;
     }
   ```




-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantCreateRequest.java
##########
@@ -141,7 +142,9 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     // A caveat is that this assumes OM's auth_to_local is the same as
     //  the client's. Maybe move this logic to the client and pass VolumeArgs?
     final String owner = ugi.getShortUserName();
-    final String volumeName = tenantId;  // TODO: Configurable
+    // Volume name defaults to tenant name if unspecified in the request
+    final String volumeName =
+        request.hasTenantName() ? request.getTenantName() : tenantId;

Review comment:
       I filed HDDS-6052 to fully implement this feature, we can do it there.




-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -94,26 +99,26 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantName) {
+      String tenantId) {
 
     final OmDBTenantInfo tenantInfo;
     try {
-      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantName);
+      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);
     } catch (IOException e) {
       throw new RuntimeException("Potential DB error. Unable to retrieve "
-          + "OmDBTenantInfo entry for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry for tenant '" + tenantId + "'.");

Review comment:
       Yup moving the call inside the try/catch on applyTransaction looks good. The call to `createErrorOMResponse` when the IOException is caught in applyTransaction will assign the response a status of `INTERNAL_ERROR` and terminate the OM as expected on DB failure.




-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
##########
@@ -408,7 +410,7 @@ public void testOzoneTenantBasicOperations() throws IOException {
     executeHA(tenantShell, new String[] {"create", "finance"});

Review comment:
       Side note: we should have a jira for the fixme above so we don't forget about it. Or change it to say TODO so we can grep and fix all the TODOs our final merge. Unrelated but I just happened to see it.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantCreateRequest.java
##########
@@ -141,7 +142,9 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     // A caveat is that this assumes OM's auth_to_local is the same as
     //  the client's. Maybe move this logic to the client and pass VolumeArgs?
     final String owner = ugi.getShortUserName();
-    final String volumeName = tenantId;  // TODO: Configurable
+    // Volume name defaults to tenant name if unspecified in the request
+    final String volumeName =
+        request.hasTenantName() ? request.getTenantName() : tenantId;

Review comment:
       Good catch.
   
   CreateTenant with custom volume name is not fully implemented yet. It is actually out of the scope of this jira (even though I have been writing it pieces by pieces as I glimpsed at the TODOs and decided to implement some part of it, as I have also done in this PR).
   
   Should probably do that in the next one, then add proper tests. And we could have more then just volumeName that can be customized.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantCreateRequest.java
##########
@@ -141,7 +142,9 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     // A caveat is that this assumes OM's auth_to_local is the same as
     //  the client's. Maybe move this logic to the client and pass VolumeArgs?
     final String owner = ugi.getShortUserName();
-    final String volumeName = tenantId;  // TODO: Configurable
+    // Volume name defaults to tenant name if unspecified in the request

Review comment:
       done




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -668,29 +670,63 @@ public void revokeS3Secret(String kerberosID) throws IOException {
    * {@inheritDoc}
    */
   @Override
-  public void createTenant(String tenantName) throws IOException {
-    Preconditions.checkArgument(Strings.isNotBlank(tenantName),
+  public void createTenant(String tenantId) throws IOException {
+    Preconditions.checkArgument(Strings.isNotBlank(tenantId),
         "tenantName cannot be null or empty.");
-    ozoneManagerClient.createTenant(tenantName);
+    ozoneManagerClient.createTenant(tenantId);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void createTenant(String tenantId, OmTenantArgs omTenantArgs)

Review comment:
       Will add `TenantArgs`. Though I am not entirely sure why the client wouldn't just use `OmVolumeArgs` for `createVolume`. It might have something to do with the extra metadata (ObjectID/UpdateID).




-- 
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] smengcl commented on pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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


   Known flaky (and unrelated) test `TestHDDSUpgrade ` (HDDS-5971). Retriggered CI.


-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/lock/OzoneManagerLock.java
##########
@@ -389,8 +392,8 @@ private void unlock(Resource resource, String resourceName,
     USER_LOCK((byte) 3, "USER_LOCK"), // 15
 
     S3_SECRET_LOCK((byte) 4, "S3_SECRET_LOCK"), // 31
-    PREFIX_LOCK((byte) 5, "PREFIX_LOCK"); //63
-//    TENANT_LOCK((byte) 6, "TENANT_LOCK"); // 127
+    PREFIX_LOCK((byte) 5, "PREFIX_LOCK"), //63
+    TENANT_LOCK((byte) 6, "TENANT_LOCK"); // 127

Review comment:
       Assign TENANT_LOCK weight 0.




-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -94,26 +99,26 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantName) {
+      String tenantId) {
 
     final OmDBTenantInfo tenantInfo;
     try {
-      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantName);
+      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);
     } catch (IOException e) {
       throw new RuntimeException("Potential DB error. Unable to retrieve "
-          + "OmDBTenantInfo entry for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry for tenant '" + tenantId + "'.");
     }
 
     if (tenantInfo == null) {
       throw new RuntimeException("Potential DB error or race condition. "
-          + "OmDBTenantInfo entry is missing for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry is missing for tenant '" + tenantId + "'.");
     }
 
-    final String volumeName = tenantInfo.getAccountNamespaceName();
+    final String volumeName = tenantInfo.getBucketNamespaceName();
 
-    if (StringUtils.isEmpty(tenantName)) {
+    if (StringUtils.isEmpty(tenantId)) {

Review comment:
       The point is not to terminate the whole OM because of a metadata issue for one tenant. METADATA_ERROR will still terminate the OM, see `OzoneManagerStateMachine#applyTransaction`.




-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-tenant.robot
##########
@@ -72,3 +76,12 @@ Secure Tenant Create Tenant Failure with Regular (non-admin) user
 Secure Tenant SetSecret Failure with Regular (non-admin) user
     ${rc}  ${output} =  Run And Return Rc And Output  ozone tenant user set-secret 'tenantone$bob' --secret=somesecret2 --export
                         Should contain   ${output}         Permission denied. Requested accessId
+
+Secure Tenant Revoke User AccessId Success
+    Run Keyword   Kinit test user     testuser     testuser.keytab
+    ${output} =         Execute          ozone tenant user revoke 'tenantone$bob'
+                        Should contain   ${output}         Revoked accessId 'tenantone$bob'.

Review comment:
       We should probably have a negative test case here too, since a bug in this would be bad. Try to use Bob's access ID and make sure it no longer works (same as above).




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -668,29 +670,63 @@ public void revokeS3Secret(String kerberosID) throws IOException {
    * {@inheritDoc}
    */
   @Override
-  public void createTenant(String tenantName) throws IOException {
-    Preconditions.checkArgument(Strings.isNotBlank(tenantName),
+  public void createTenant(String tenantId) throws IOException {
+    Preconditions.checkArgument(Strings.isNotBlank(tenantId),
         "tenantName cannot be null or empty.");
-    ozoneManagerClient.createTenant(tenantName);
+    ozoneManagerClient.createTenant(tenantId);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void createTenant(String tenantId, OmTenantArgs omTenantArgs)

Review comment:
       Good point.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
##########
@@ -84,6 +97,14 @@ private OmVolumeArgs(String adminName, String ownerName, String volume,
     this.updateID = updateID;
   }
 
+  public long getRefCount() {
+    assert (refCount >= 0);
+    return refCount;
+  }
+
+  public void setRefCount(long refCount) {

Review comment:
       Good idea.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantCreateRequest.java
##########
@@ -141,7 +142,9 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     // A caveat is that this assumes OM's auth_to_local is the same as
     //  the client's. Maybe move this logic to the client and pass VolumeArgs?
     final String owner = ugi.getShortUserName();
-    final String volumeName = tenantId;  // TODO: Configurable
+    // Volume name defaults to tenant name if unspecified in the request
+    final String volumeName =
+        request.hasTenantName() ? request.getTenantName() : tenantId;

Review comment:
       Good catch.
   
   CreateTenant with custom volume name is not fully implemented yet. It is actually out of the scope of this jira (even though I have being writing it pieces by pieces as I glimpsed at the TODOs and decided to implement some part of it, as I have also done in this PR). Should probably do that in the next one, then add proper tests.




-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
##########
@@ -82,8 +94,36 @@ private OmVolumeArgs(String adminName, String ownerName, String volume,
     this.modificationTime = modificationTime;
     this.objectID = objectID;
     this.updateID = updateID;
+    this.refCount = refCount;
+  }
+
+  public long getRefCount() {
+    Preconditions.checkState(refCount >= 0, "refCount should not be negative");

Review comment:
       This check should be moved to the decrement operation, where the refCount value actually changes. Otherwise we will not know where the actual error occurred.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
##########
@@ -84,6 +97,14 @@ private OmVolumeArgs(String adminName, String ownerName, String volume,
     this.updateID = updateID;
   }
 
+  public long getRefCount() {
+    assert (refCount >= 0);
+    return refCount;
+  }
+
+  public void setRefCount(long refCount) {

Review comment:
       Is there any reason to leave the setRefCount here? Still seems error prone, lots of people don't read javadocs before calling things : ) If we have a use case now we can leave it but this doesn't seem like a good thing to put in "just in case".

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
##########
@@ -444,6 +471,14 @@ public void deleteBucket(String bucketName) throws IOException {
     proxy.deleteBucket(name, bucketName);
   }
 
+  public long getRefCount() {

Review comment:
       The getter is probably ok, but we should be able to get rid of the setter without findbugs complaining because the new constructor initializes it. Probably also a good idea to have the old constructor also set it to 0 so there is no confusion about what its default value should be.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -668,29 +670,63 @@ public void revokeS3Secret(String kerberosID) throws IOException {
    * {@inheritDoc}
    */
   @Override
-  public void createTenant(String tenantName) throws IOException {
-    Preconditions.checkArgument(Strings.isNotBlank(tenantName),
+  public void createTenant(String tenantId) throws IOException {
+    Preconditions.checkArgument(Strings.isNotBlank(tenantId),
         "tenantName cannot be null or empty.");
-    ozoneManagerClient.createTenant(tenantName);
+    ozoneManagerClient.createTenant(tenantId);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void createTenant(String tenantId, OmTenantArgs omTenantArgs)

Review comment:
       Neither of them are protobuf classes, but the convention for bucket and volume creation is to not use the same class for client configuration and OM representation. I think we should be following the same pattern as existing requests like bucket and volume create here.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,206 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    return getOmRequest().toBuilder().setUserInfo(getUserInfo()).build();
   }
 
   @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    String volumeName = null;
+    boolean decVolumeRefCount = true;
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs = null;
+
+    try {
+      // Check tenant existence in tenantStateTable
+      if (!omMetadataManager.getTenantStateTable().isExist(tenantId)) {
+        LOG.debug("tenant: {} does not exist", tenantId);
+        throw new OMException("Tenant '" + tenantId + "' does not exist",
+            TENANT_NOT_FOUND);
+      }
+
+      // Reading the TenantStateTable without lock as we don't have or need
+      // a TENANT_LOCK. The assumption is that OmDBTenantInfo is read-only
+      // once it is set during tenant creation.
+      final OmDBTenantInfo dbTenantInfo =
+          omMetadataManager.getTenantStateTable().get(tenantId);
+      volumeName = dbTenantInfo.getBucketNamespaceName();
+      assert(volumeName != null);
+
+      LOG.debug("Tenant '{}' has volume '{}'", tenantId, volumeName);
+      // decVolumeRefCount is true if volumeName is not empty string
+      decVolumeRefCount = volumeName.length() > 0;
+
+      // Acquire the volume lock
+      Preconditions.checkNotNull(volumeName,

Review comment:
       This really is an assertion. So that in some other potential buggy write path we could catch it asap (instead of propagating it further).
   
   Also I just realized I already had a `assert(volumeName != null)` earlier. So I'm just going to remove this line. (though assert keyword doesn't trigger in prod)




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -94,26 +99,26 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantName) {
+      String tenantId) {
 
     final OmDBTenantInfo tenantInfo;
     try {
-      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantName);
+      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);
     } catch (IOException e) {
       throw new RuntimeException("Potential DB error. Unable to retrieve "
-          + "OmDBTenantInfo entry for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry for tenant '" + tenantId + "'.");
     }
 
     if (tenantInfo == null) {
       throw new RuntimeException("Potential DB error or race condition. "
-          + "OmDBTenantInfo entry is missing for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry is missing for tenant '" + tenantId + "'.");
     }
 
-    final String volumeName = tenantInfo.getAccountNamespaceName();
+    final String volumeName = tenantInfo.getBucketNamespaceName();
 
-    if (StringUtils.isEmpty(tenantName)) {
+    if (StringUtils.isEmpty(tenantId)) {

Review comment:
       Done. but throwing `METADATA_ERROR` instead. The `volumeName` field is not supposed to be null(or "")




-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,256 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.METADATA_ERROR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IS_REFERENCED;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.TENANT_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    // Check if there are any accessIds in the tenant
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getBucketNamespaceName();
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be removed?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false;
+    boolean acquiredUserLock = false;
+    boolean acquiredTenantLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string

Review comment:
       Is this still the case? I thought the client was explicitly setting the volume name every time now.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,256 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.METADATA_ERROR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IS_REFERENCED;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.TENANT_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    // Check if there are any accessIds in the tenant
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {

Review comment:
       This should be moved to validateAndUpdateCache as well, since this call will need to check DB state.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,256 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.METADATA_ERROR;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IS_REFERENCED;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.TENANT_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {

Review comment:
       This is not safe. preExecute is not serialized, so just because the tenant exists here, it may not exist when we do the get on line 103. We do need to know the volume for this tenant, which requires reading the DB, so I think the flow here should be refactored: Instead of appending a delete volume request to the om request in preExecute, we can construct that same info in validateAndUpdateCache and use that to delete the volume.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantCreateHandler.java
##########
@@ -33,26 +32,19 @@
     description = "Create one or more tenants")
 public class TenantCreateHandler extends TenantHandler {
 
-  @CommandLine.Spec
-  private CommandLine.Model.CommandSpec spec;
-
-  @CommandLine.Parameters(description = "List of tenant names")
+  @CommandLine.Parameters(description = "List of tenant names", arity = "1..")
   private List<String> tenants = new ArrayList<>();
 
   @Override
   protected void execute(OzoneClient client, OzoneAddress address) {
-    if (tenants.size() > 0) {
-      for (String tenantName : tenants) {
-        try {
-          client.getObjectStore().createTenant(tenantName);
-          out().println("Created tenant '" + tenantName + "'.");
-        } catch (IOException e) {
-          err().println("Failed to create tenant '" + tenantName + "': " +
-              e.getMessage());
-        }
+    for (String tenantId : tenants) {

Review comment:
       I think I will file a jira to remove the loop for create/delete tenant CLI.
   
   But this might actually have value for assign user to tenant? e.g. An admin batch assigning users to a tenant without waiting for seconds every time the Ozone CLI client's JVM boots up.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // TODO: Check tenantId validity? Maybe not
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    // Check if the tenant volume is empty (when the volume exists)
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getAssociatedVolumeName();
+    // Note: Looks like isVolumeEmpty() returns true if volume doesn't exist
+    if (volumeName.length() > 0 && !metadataManager.isVolumeEmpty(volumeName)) {
+      LOG.debug("volume: {} is not empty", volumeName);
+      throw new OMException("Tenant volume '" + volumeName + "' is not empty." +
+          " Volume must be emptied before the tenant can be deleted.",
+          VOLUME_NOT_EMPTY);
+    }
+
+    // Check if there are any accessIds in the tenant
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be ignored?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  public void handleRequestFailure(OzoneManager ozoneManager) {
+    super.handleRequestFailure(ozoneManager);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false, acquiredUserLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string
+    final String volumeName =
+        getOmRequest().getDeleteVolumeRequest().getVolumeName();
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs;
+    String volumeOwner = null;
+    // deleteVolume is true if volumeName is not empty string
+    boolean deleteVolume = volumeName.length() > 0;
+    OzoneManagerStorageProtos.PersistedUserVolumeInfo newVolumeList = null;
+
+    try {
+      if (deleteVolume) {
+        // check Acl
+        if (ozoneManager.getAclsEnabled()) {
+          checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+              OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+              volumeName, null, null);
+        }
+
+        acquiredVolumeLock = omMetadataManager.getLock().acquireWriteLock(
+            VOLUME_LOCK, volumeName);
+
+        omVolumeArgs = getVolumeInfo(omMetadataManager, volumeName);
+        // Check volume ref count
+        long volRefCount = omVolumeArgs.getRefCount();
+        if (volRefCount > 1L) {
+          LOG.warn("Volume '{}' has a greater than 1 reference count of " +
+                  "'{}'", volumeName, volRefCount);
+          throw new OMException("Volume '" + volumeName + "' has a greater " +
+              "than 1 reference count of '" + volRefCount + "'. This may " +
+              "indicate the volume is in use by some other Ozone features. " +
+              "Please disable such other features before trying to delete " +
+              "the tenant again.", VOLUME_IN_USE);
+        }
+
+        volumeOwner = omVolumeArgs.getOwnerName();
+        acquiredUserLock = omMetadataManager.getLock().acquireWriteLock(
+            USER_LOCK, volumeOwner);
+
+        // Check volume emptiness, again
+        if (!omMetadataManager.isVolumeEmpty(volumeName)) {
+          LOG.debug("volume: '{}' is not empty", volumeName);
+          throw new OMException("Aborting tenant deletion. " +
+              "Volume becomes non-empty somewhere between" +
+              "preExecute and validateAndUpdateCache", VOLUME_NOT_EMPTY);
+        }
+
+        // Actual volume deletion, follows OMVolumeDeleteRequest
+        newVolumeList =
+            omMetadataManager.getUserTable().get(volumeOwner);
+        newVolumeList = delVolumeFromOwnerList(newVolumeList, volumeName,
+            volumeOwner, transactionLogIndex);
+        final String dbUserKey = omMetadataManager.getUserKey(volumeOwner);
+        omMetadataManager.getUserTable().addCacheEntry(
+            new CacheKey<>(dbUserKey),
+            new CacheValue<>(Optional.of(newVolumeList), transactionLogIndex));
+        final String dbVolumeKey = omMetadataManager.getVolumeKey(volumeName);
+        omMetadataManager.getVolumeTable().addCacheEntry(
+            new CacheKey<>(dbVolumeKey),
+            new CacheValue<>(Optional.absent(), transactionLogIndex));
+
+        // TODO: Set response dbVolumeKey?
+      }
+
+      // TODO: Should hold some tenant lock here. Just in case !deleteVolume
+//      acquiredTenantLock = omMetadataManager.getLock().acquireWriteLock(
+//          TENANT_LOCK, tenantId);

Review comment:
       Hi @errose28 , actually I have implemented `TENANT_LOCK`.
   
   But there's the caveat of lock ordering (See `OzoneManagerLock` class header). I assigned weight `6` to `TENANT_LOCK`. Therefore, I have to release `TENANT_LOCK` before grabbing `VOLUME_LOCK` (weight `0`). This means there could be a gap where `TENANT_LOCK` is released but `VOLUME_LOCK` is not yet acquired. For example, here: 
   
   https://github.com/apache/ozone/blob/882809889c79bd45ff908d9ccc312859861798c9/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java#L152-L170
   
   Not sure if this could cause any real issues. I think not?
   
   1. Or sometimes I could first grab `VOLUME_LOCK`, then grab `TENANT_LOCK`, but that's not always viable, as shown in the above example. In this case it has to get volume name from tenant table before it could grab the `VOLUME_LOCK`.
   
   2. Another solution might be to move `TENANT_LOCK` weight ahead of `VOLUME_LOCK`. i.e. making `TENANT_LOCK` weight `0`. Not sure we want to do that. We could discuss.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
##########
@@ -344,7 +344,7 @@ public void testAssignAdmin() throws IOException {
     checkOutput(err, "", true);
 
     // Loop assign-revoke 3 times
-    for (int i = 0; i < 3; i++) {
+    for (int i = 0; i < 1; i++) {

Review comment:
       Oops. Changed for testing. Staged this line by mistake.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // TODO: Check tenantId validity? Maybe not
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    // Check if the tenant volume is empty (when the volume exists)
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getAssociatedVolumeName();
+    // Note: Looks like isVolumeEmpty() returns true if volume doesn't exist
+    if (volumeName.length() > 0 && !metadataManager.isVolumeEmpty(volumeName)) {
+      LOG.debug("volume: {} is not empty", volumeName);
+      throw new OMException("Tenant volume '" + volumeName + "' is not empty." +
+          " Volume must be emptied before the tenant can be deleted.",
+          VOLUME_NOT_EMPTY);
+    }
+
+    // Check if there are any accessIds in the tenant
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be ignored?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  public void handleRequestFailure(OzoneManager ozoneManager) {
+    super.handleRequestFailure(ozoneManager);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false, acquiredUserLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string
+    final String volumeName =
+        getOmRequest().getDeleteVolumeRequest().getVolumeName();
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs;
+    String volumeOwner = null;
+    // deleteVolume is true if volumeName is not empty string
+    boolean deleteVolume = volumeName.length() > 0;
+    OzoneManagerStorageProtos.PersistedUserVolumeInfo newVolumeList = null;
+
+    try {
+      if (deleteVolume) {
+        // check Acl
+        if (ozoneManager.getAclsEnabled()) {
+          checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+              OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+              volumeName, null, null);
+        }
+
+        acquiredVolumeLock = omMetadataManager.getLock().acquireWriteLock(
+            VOLUME_LOCK, volumeName);
+
+        omVolumeArgs = getVolumeInfo(omMetadataManager, volumeName);
+        // Check volume ref count
+        long volRefCount = omVolumeArgs.getRefCount();
+        if (volRefCount > 1L) {
+          LOG.warn("Volume '{}' has a greater than 1 reference count of " +
+                  "'{}'", volumeName, volRefCount);
+          throw new OMException("Volume '" + volumeName + "' has a greater " +
+              "than 1 reference count of '" + volRefCount + "'. This may " +
+              "indicate the volume is in use by some other Ozone features. " +
+              "Please disable such other features before trying to delete " +
+              "the tenant again.", VOLUME_IN_USE);
+        }
+
+        volumeOwner = omVolumeArgs.getOwnerName();
+        acquiredUserLock = omMetadataManager.getLock().acquireWriteLock(
+            USER_LOCK, volumeOwner);
+
+        // Check volume emptiness, again
+        if (!omMetadataManager.isVolumeEmpty(volumeName)) {
+          LOG.debug("volume: '{}' is not empty", volumeName);
+          throw new OMException("Aborting tenant deletion. " +
+              "Volume becomes non-empty somewhere between" +
+              "preExecute and validateAndUpdateCache", VOLUME_NOT_EMPTY);
+        }
+
+        // Actual volume deletion, follows OMVolumeDeleteRequest
+        newVolumeList =
+            omMetadataManager.getUserTable().get(volumeOwner);
+        newVolumeList = delVolumeFromOwnerList(newVolumeList, volumeName,
+            volumeOwner, transactionLogIndex);
+        final String dbUserKey = omMetadataManager.getUserKey(volumeOwner);
+        omMetadataManager.getUserTable().addCacheEntry(
+            new CacheKey<>(dbUserKey),
+            new CacheValue<>(Optional.of(newVolumeList), transactionLogIndex));
+        final String dbVolumeKey = omMetadataManager.getVolumeKey(volumeName);
+        omMetadataManager.getVolumeTable().addCacheEntry(
+            new CacheKey<>(dbVolumeKey),
+            new CacheValue<>(Optional.absent(), transactionLogIndex));
+
+        // TODO: Set response dbVolumeKey?
+      }
+
+      // TODO: Should hold some tenant lock here. Just in case !deleteVolume
+//      acquiredTenantLock = omMetadataManager.getLock().acquireWriteLock(
+//          TENANT_LOCK, tenantId);

Review comment:
       We haven't decided on whether we want the `TENANT_LOCK` or not yet. Last time I talked to @prashantpogde and we didn't think it is needed but it might become necessary here. Or a coarse-gained global-ish lock on the would also work as tenant creation/deletion is not intended to be a high-frequency operation (as long as the lock doesn't block other high-frequency requests).




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantCreateRequest.java
##########
@@ -141,7 +142,9 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     // A caveat is that this assumes OM's auth_to_local is the same as
     //  the client's. Maybe move this logic to the client and pass VolumeArgs?
     final String owner = ugi.getShortUserName();
-    final String volumeName = tenantId;  // TODO: Configurable
+    // Volume name defaults to tenant name if unspecified in the request
+    final String volumeName =
+        request.hasTenantName() ? request.getTenantName() : tenantId;

Review comment:
       Good catch.
   
   CreateTenant with custom volume name is not fully implemented yet. It is actually out of the scope of this jira (even though I have been writing it pieces by pieces as I glimpsed at the TODOs and decided to implement some part of it, as I have also done in this PR).
   
   Should probably do that in the next one, then add proper tests. And it should have more then just volumeName that can be customized.




-- 
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] smengcl edited a comment on pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

Posted by GitBox <gi...@apache.org>.
smengcl edited a comment on pull request #2857:
URL: https://github.com/apache/ozone/pull/2857#issuecomment-977343073


   > Thanks for working on this @smengcl. Large PR so lots of comments : ) At a high level there are a few things I'd like to mention:
   > 
   > 1. I didn't see much testing of tenant delete beyond just that the command succeeded. Let me know if I missed something, but it looks like we need more in depth testing of the command to check that volume is removed, remaining tenant state is cleared up, etc.
   > 2. I think we should consider the ref count idea a bit more carefully. Right now nothing decrements the count, so if someone else uses it for something and does the same thing we did the volume will be stuck forever. Decrementing the count for us would imply an alternate tenant delete variation which deletes the tenant state but not the volume, decrementing the count. I think this might be a good thing to have but we should discuss.
   
   Thanks @errose28 for the review.
   
   1. There is only one negative test case in the integration test, that checks the tenant can't be removed if there are accessIds assigned: https://github.com/apache/ozone/blob/3d689c924ae27d855f818be9c3f90a000674e81f/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java#L563-L568
   
   And one for acceptance test: https://github.com/apache/ozone/blob/3d689c924ae27d855f818be9c3f90a000674e81f/hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-tenant.robot#L63-L65
   
   No tests for volume emptiness / volume existence negative tests have been added yet.
   
   2. Yes. In an earlier unpublished iteration I do decrease the volume refCount, but that logic was later replaced by the thrown exception. Since how `DeleteTenantRequest` should behave when `refCount > 1L` is unclear here. (Should we just keep the volume and only decrease the refCount? Probably no, because the Ranger policies will be removed regardless, which renders the volume inaccessible to all users but cluster admins). So the assumption made here is that the volume `refCount` MUST be `<=1` (or == 1 to be exact) to pass this check. -- Essentially all other new features in the future that could increment volume `refCount` must be disabled on the volume first, only then the cluster admin can do `tenant delete`. We can discuss it next time.


-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantCreateRequest.java
##########
@@ -141,7 +142,9 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     // A caveat is that this assumes OM's auth_to_local is the same as
     //  the client's. Maybe move this logic to the client and pass VolumeArgs?
     final String owner = ugi.getShortUserName();
-    final String volumeName = tenantId;  // TODO: Configurable
+    // Volume name defaults to tenant name if unspecified in the request
+    final String volumeName =
+        request.hasTenantName() ? request.getTenantName() : tenantId;

Review comment:
       Good catch.
   
   CreateTenant with custom volume name is not fully implemented yet. It is actually out of the scope of this jira (even though I have been writing it pieces by pieces as I glimpsed at the TODOs and decided to implement some part of it, as I have also done in this PR). Should probably do that in the next one, then add proper tests.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantCreateRequest.java
##########
@@ -141,7 +142,9 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     // A caveat is that this assumes OM's auth_to_local is the same as
     //  the client's. Maybe move this logic to the client and pass VolumeArgs?
     final String owner = ugi.getShortUserName();
-    final String volumeName = tenantId;  // TODO: Configurable
+    // Volume name defaults to tenant name if unspecified in the request
+    final String volumeName =
+        request.hasTenantName() ? request.getTenantName() : tenantId;

Review comment:
       Good catch. Yep, CreateTenant with custom volume name is not fully implemented yet.




-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -668,29 +670,63 @@ public void revokeS3Secret(String kerberosID) throws IOException {
    * {@inheritDoc}
    */
   @Override
-  public void createTenant(String tenantName) throws IOException {
-    Preconditions.checkArgument(Strings.isNotBlank(tenantName),
+  public void createTenant(String tenantId) throws IOException {
+    Preconditions.checkArgument(Strings.isNotBlank(tenantId),
         "tenantName cannot be null or empty.");
-    ozoneManagerClient.createTenant(tenantName);
+    ozoneManagerClient.createTenant(tenantId);
+  }
+
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void createTenant(String tenantId, OmTenantArgs omTenantArgs)

Review comment:
       Looking at RpcClient#createVolume I think the convention is that things in the ozone.client package are passed to the RpcClient, which then converts them to the corresponding type from om.helpers package before passing to the RPC call. We should probably follow this convention here.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
##########
@@ -84,6 +97,14 @@ private OmVolumeArgs(String adminName, String ownerName, String volume,
     this.updateID = updateID;
   }
 
+  public long getRefCount() {
+    assert (refCount >= 0);

Review comment:
       nit. We can use the Preconditions library with a message for this check to be consistent with existing code.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantCreateRequest.java
##########
@@ -141,7 +142,9 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     // A caveat is that this assumes OM's auth_to_local is the same as
     //  the client's. Maybe move this logic to the client and pass VolumeArgs?
     final String owner = ugi.getShortUserName();
-    final String volumeName = tenantId;  // TODO: Configurable
+    // Volume name defaults to tenant name if unspecified in the request
+    final String volumeName =
+        request.hasTenantName() ? request.getTenantName() : tenantId;

Review comment:
       Reading from the CreateTenantRequest proto here, looks like this should be
   ```
   final String volumeName =
           request.hasVolumeName() ? request.getVolumeName() : tenantId;
   ```
   Also this reveals there is no testing for tenantName != volumeName, so we should probably add that in this PR since its adding the volumeName proto field.

##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -416,6 +416,9 @@ enum Status {
     INVALID_TENANT_USER_NAME = 80;
     INVALID_ACCESSID = 81;
     TENANT_AUTHORIZER_ERROR = 82;
+
+    VOLUME_IN_USE = 83;
+    TENANT_NOT_EMPTY = 84;  // TODO: Renumber this when rebasing?

Review comment:
       nit. Don't need this TODO. protoc will fail if the IDs conflict after rebase/merge from master.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/volume/OMVolumeDeleteRequest.java
##########
@@ -100,6 +100,14 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
 
       OmVolumeArgs omVolumeArgs = getVolumeInfo(omMetadataManager, volume);
 
+      // Check reference count
+      if (omVolumeArgs.getRefCount() != 0L) {
+        LOG.debug("volume: {} has non-zero ref count. won't delete", volume);
+        throw new OMException("Volume is being used. Use " +
+            "`ozone tenant delete` CLI to delete the volume instead.",
+            OMException.ResultCodes.VOLUME_IN_USE);

Review comment:
       Maybe we could say "volume is referenced" instead of "volume is in use" since we are calling the new field a reference count.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmDBTenantInfo.java
##########
@@ -117,6 +117,13 @@ public String getBucketPolicyGroupName() {
     return bucketPolicyGroupName;
   }
 
+  // NOTE: Should return an empty string "" if somehow the tenant is not
+  // associated with a volume. Never return null.
+  public String getAssociatedVolumeName() {

Review comment:
       What's the distinction between this and the bucket namespace? Does it need to be a different method?

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmTenantArgs.java
##########
@@ -22,14 +22,26 @@
  * This class is used for storing Ozone tenant arguments.
  */
 public class OmTenantArgs {
-  /* Tenant name */
+  /**
+   * Tenant name.
+   */
   private final String tenantId;
+  /**
+   * Volume name to be created for this tenant.
+   * Default volume name would be the same as tenant name if unspecified.

Review comment:
       Can we do this with a constructor overload instead of leaving it up to interpretation of whoever uses this class?

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
##########
@@ -408,7 +410,7 @@ public void testOzoneTenantBasicOperations() throws IOException {
     executeHA(tenantShell, new String[] {"create", "finance"});

Review comment:
       Side note: we should have a jira for the fixme above so we don't forget about it.

##########
File path: hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-tenant.robot
##########
@@ -60,6 +60,14 @@ Secure Tenant GetSecret Success
     ${output} =         Execute          ozone tenant user getsecret 'tenantone$bob' --export
                         Should contain   ${output}         export AWS_SECRET_ACCESS_KEY='somesecret1'
 
+Secure Tenant Delete Tenant Failure Tenant Not Empty
+    ${rc}  ${output} =  Run And Return Rc And Output  ozone tenant delete tenantone
+                        Should contain   ${output}         Failed to delete tenant 'tenantone': Tenant 'tenantone' is not empty
+
+Secure Tenant Revoke User AccessId Success
+    ${output} =         Execute          ozone tenant user revoke 'tenantone$bob'
+                        Should contain   ${output}         Revoked accessId 'tenantone$bob'.

Review comment:
       Probably want a negative test case here, since a bug would be very bad. Try to use Bob's access ID and make sure it doesn't work.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
##########
@@ -444,6 +471,14 @@ public void deleteBucket(String bucketName) throws IOException {
     proxy.deleteBucket(name, bucketName);
   }
 
+  public long getRefCount() {

Review comment:
       This class is returned to the client. Do we want to expose ref count to the client? I think at least the setter should not be here, but I'm not sure whether or not we want the getter.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
##########
@@ -84,6 +97,14 @@ private OmVolumeArgs(String adminName, String ownerName, String volume,
     this.updateID = updateID;
   }
 
+  public long getRefCount() {
+    assert (refCount >= 0);
+    return refCount;
+  }
+
+  public void setRefCount(long refCount) {

Review comment:
       This should be an increment/decrement so requests cannot (accidentally) overwrite the current volume's ref count.

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
##########
@@ -591,7 +622,14 @@ public void testListTenantUsers() throws IOException {
     checkOutput(out, "", true);
     checkOutput(err, "Revoked accessId", false);
 
-    // TODO: Clean up: remove tenant when tenant remove CLI is implemented
+    executeHA(tenantShell, new String[] {
+        "user", "revoke", "tenant1$bob"});
+    checkOutput(out, "", true);
+    checkOutput(err, "Revoked accessId", false);
+
+    executeHA(tenantShell, new String[] {"delete", "tenant1"});
+    checkOutput(out, "Deleted tenant 'tenant1'.\n", true);
+    checkOutput(err, "", true);

Review comment:
       Since we are testing list tenant here, a list after the delete for this test might be good too.

##########
File path: hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-tenant.robot
##########
@@ -72,3 +80,8 @@ Secure Tenant Create Tenant Failure with Regular (non-admin) user
 Secure Tenant SetSecret Failure with Regular (non-admin) user
     ${rc}  ${output} =  Run And Return Rc And Output  ozone tenant user set-secret 'tenantone$bob' --secret=somesecret2 --export
                         Should contain   ${output}         Permission denied. Requested accessId
+
+Secure Tenant Delete Tenant Success
+    Run Keyword   Kinit test user     testuser     testuser.keytab
+    ${output} =         Execute          ozone tenant delete tenantone
+                        Should contain   ${output}         Deleted tenant 'tenantone'.

Review comment:
       A negative test might be good here too. Like following this with a tenant list command and making sure the tenant is actually gone.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+

Review comment:
       Why are we checking all this DB state in preExecute? Minor performance optimization? We just have to check it all again in validateAndUpdateCache if these checks pass. Is this pattern used elsewhere for existing OM requests?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantCreateRequest.java
##########
@@ -141,7 +142,9 @@ public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
     // A caveat is that this assumes OM's auth_to_local is the same as
     //  the client's. Maybe move this logic to the client and pass VolumeArgs?
     final String owner = ugi.getShortUserName();
-    final String volumeName = tenantId;  // TODO: Configurable
+    // Volume name defaults to tenant name if unspecified in the request

Review comment:
       Similar to above, do we want to handle this on the client side? We could have the volume name always be set, and it's up to the client whether or not that is the same as the tenant name?

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/shell/TestOzoneTenantShell.java
##########
@@ -344,7 +344,7 @@ public void testAssignAdmin() throws IOException {
     checkOutput(err, "", true);
 
     // Loop assign-revoke 3 times
-    for (int i = 0; i < 3; i++) {
+    for (int i = 0; i < 1; i++) {

Review comment:
       Why the one iteration for loop?

##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -463,6 +466,7 @@ message VolumeInfo {
     optional uint64 modificationTime = 10;
     optional int64 quotaInNamespace = 11 [default = -2];
     optional uint64 usedNamespace = 12;
+    optional int64 refCount = 13;

Review comment:
       Same as above. If we decide not to expose ref count to the client we can remove this.

##########
File path: hadoop-ozone/dist/src/main/smoketest/security/ozone-secure-tenant.robot
##########
@@ -72,3 +76,12 @@ Secure Tenant Create Tenant Failure with Regular (non-admin) user
 Secure Tenant SetSecret Failure with Regular (non-admin) user
     ${rc}  ${output} =  Run And Return Rc And Output  ozone tenant user set-secret 'tenantone$bob' --secret=somesecret2 --export
                         Should contain   ${output}         Permission denied. Requested accessId
+
+Secure Tenant Revoke User AccessId Success
+    Run Keyword   Kinit test user     testuser     testuser.keytab
+    ${output} =         Execute          ozone tenant user revoke 'tenantone$bob'
+                        Should contain   ${output}         Revoked accessId 'tenantone$bob'.

Review comment:
       We should probably have a negative test case here too, since a bug in this would be bad. Try to use Bob's access ID and make sure it no longer works.

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantCreateHandler.java
##########
@@ -33,26 +32,19 @@
     description = "Create one or more tenants")
 public class TenantCreateHandler extends TenantHandler {
 
-  @CommandLine.Spec
-  private CommandLine.Model.CommandSpec spec;
-
-  @CommandLine.Parameters(description = "List of tenant names")
+  @CommandLine.Parameters(description = "List of tenant names", arity = "1..")
   private List<String> tenants = new ArrayList<>();
 
   @Override
   protected void execute(OzoneClient client, OzoneAddress address) {
-    if (tenants.size() > 0) {
-      for (String tenantName : tenants) {
-        try {
-          client.getObjectStore().createTenant(tenantName);
-          out().println("Created tenant '" + tenantName + "'.");
-        } catch (IOException e) {
-          err().println("Failed to create tenant '" + tenantName + "': " +
-              e.getMessage());
-        }
+    for (String tenantId : tenants) {

Review comment:
       Do we have other CLIs like this that issue multiple OM requests? If nobody else is doing this, I would say we avoid it. It's confusing to the user what state is created if one of the multiple requests fails.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // TODO: Check tenantId validity? Maybe not
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    // Check if the tenant volume is empty (when the volume exists)
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getAssociatedVolumeName();
+    // Note: Looks like isVolumeEmpty() returns true if volume doesn't exist
+    if (volumeName.length() > 0 && !metadataManager.isVolumeEmpty(volumeName)) {
+      LOG.debug("volume: {} is not empty", volumeName);
+      throw new OMException("Tenant volume '" + volumeName + "' is not empty." +
+          " Volume must be emptied before the tenant can be deleted.",
+          VOLUME_NOT_EMPTY);
+    }
+
+    // Check if there are any accessIds in the tenant
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be ignored?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  public void handleRequestFailure(OzoneManager ozoneManager) {
+    super.handleRequestFailure(ozoneManager);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false, acquiredUserLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string
+    final String volumeName =
+        getOmRequest().getDeleteVolumeRequest().getVolumeName();
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs;
+    String volumeOwner = null;
+    // deleteVolume is true if volumeName is not empty string
+    boolean deleteVolume = volumeName.length() > 0;
+    OzoneManagerStorageProtos.PersistedUserVolumeInfo newVolumeList = null;
+
+    try {
+      if (deleteVolume) {
+        // check Acl
+        if (ozoneManager.getAclsEnabled()) {
+          checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+              OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+              volumeName, null, null);
+        }
+
+        acquiredVolumeLock = omMetadataManager.getLock().acquireWriteLock(
+            VOLUME_LOCK, volumeName);
+
+        omVolumeArgs = getVolumeInfo(omMetadataManager, volumeName);
+        // Check volume ref count
+        long volRefCount = omVolumeArgs.getRefCount();
+        if (volRefCount > 1L) {
+          LOG.warn("Volume '{}' has a greater than 1 reference count of " +
+                  "'{}'", volumeName, volRefCount);
+          throw new OMException("Volume '" + volumeName + "' has a greater " +
+              "than 1 reference count of '" + volRefCount + "'. This may " +
+              "indicate the volume is in use by some other Ozone features. " +
+              "Please disable such other features before trying to delete " +
+              "the tenant again.", VOLUME_IN_USE);
+        }
+
+        volumeOwner = omVolumeArgs.getOwnerName();
+        acquiredUserLock = omMetadataManager.getLock().acquireWriteLock(
+            USER_LOCK, volumeOwner);
+
+        // Check volume emptiness, again
+        if (!omMetadataManager.isVolumeEmpty(volumeName)) {
+          LOG.debug("volume: '{}' is not empty", volumeName);
+          throw new OMException("Aborting tenant deletion. " +
+              "Volume becomes non-empty somewhere between" +
+              "preExecute and validateAndUpdateCache", VOLUME_NOT_EMPTY);
+        }
+
+        // Actual volume deletion, follows OMVolumeDeleteRequest
+        newVolumeList =
+            omMetadataManager.getUserTable().get(volumeOwner);
+        newVolumeList = delVolumeFromOwnerList(newVolumeList, volumeName,
+            volumeOwner, transactionLogIndex);
+        final String dbUserKey = omMetadataManager.getUserKey(volumeOwner);
+        omMetadataManager.getUserTable().addCacheEntry(
+            new CacheKey<>(dbUserKey),
+            new CacheValue<>(Optional.of(newVolumeList), transactionLogIndex));
+        final String dbVolumeKey = omMetadataManager.getVolumeKey(volumeName);
+        omMetadataManager.getVolumeTable().addCacheEntry(
+            new CacheKey<>(dbVolumeKey),
+            new CacheValue<>(Optional.absent(), transactionLogIndex));
+
+        // TODO: Set response dbVolumeKey?
+      }
+
+      // TODO: Should hold some tenant lock here. Just in case !deleteVolume
+//      acquiredTenantLock = omMetadataManager.getLock().acquireWriteLock(
+//          TENANT_LOCK, tenantId);
+
+      // Double check tenant emptiness
+
+      // Invalidate cache entries for tenant
+      omMetadataManager.getTenantStateTable().addCacheEntry(
+          new CacheKey<>(tenantId),
+          new CacheValue<>(Optional.absent(), transactionLogIndex));
+      final String userPolicyGroupName =
+          tenantId + OzoneConsts.DEFAULT_TENANT_USER_POLICY_SUFFIX;
+      final String bucketPolicyGroupName =
+          tenantId + OzoneConsts.DEFAULT_TENANT_BUCKET_POLICY_SUFFIX;

Review comment:
       Can we do the concatenation to create the keys with a helper method in OMTenantRequestHelper? Less error prone for other devs to work with.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // TODO: Check tenantId validity? Maybe not
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    // Check if the tenant volume is empty (when the volume exists)
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getAssociatedVolumeName();
+    // Note: Looks like isVolumeEmpty() returns true if volume doesn't exist
+    if (volumeName.length() > 0 && !metadataManager.isVolumeEmpty(volumeName)) {
+      LOG.debug("volume: {} is not empty", volumeName);
+      throw new OMException("Tenant volume '" + volumeName + "' is not empty." +
+          " Volume must be emptied before the tenant can be deleted.",
+          VOLUME_NOT_EMPTY);
+    }
+
+    // Check if there are any accessIds in the tenant
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be ignored?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  public void handleRequestFailure(OzoneManager ozoneManager) {
+    super.handleRequestFailure(ozoneManager);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false, acquiredUserLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string
+    final String volumeName =
+        getOmRequest().getDeleteVolumeRequest().getVolumeName();
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs;
+    String volumeOwner = null;
+    // deleteVolume is true if volumeName is not empty string
+    boolean deleteVolume = volumeName.length() > 0;
+    OzoneManagerStorageProtos.PersistedUserVolumeInfo newVolumeList = null;
+
+    try {
+      if (deleteVolume) {
+        // check Acl
+        if (ozoneManager.getAclsEnabled()) {
+          checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+              OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+              volumeName, null, null);
+        }
+
+        acquiredVolumeLock = omMetadataManager.getLock().acquireWriteLock(
+            VOLUME_LOCK, volumeName);
+
+        omVolumeArgs = getVolumeInfo(omMetadataManager, volumeName);
+        // Check volume ref count
+        long volRefCount = omVolumeArgs.getRefCount();
+        if (volRefCount > 1L) {
+          LOG.warn("Volume '{}' has a greater than 1 reference count of " +
+                  "'{}'", volumeName, volRefCount);
+          throw new OMException("Volume '" + volumeName + "' has a greater " +
+              "than 1 reference count of '" + volRefCount + "'. This may " +
+              "indicate the volume is in use by some other Ozone features. " +
+              "Please disable such other features before trying to delete " +
+              "the tenant again.", VOLUME_IN_USE);
+        }
+
+        volumeOwner = omVolumeArgs.getOwnerName();
+        acquiredUserLock = omMetadataManager.getLock().acquireWriteLock(
+            USER_LOCK, volumeOwner);
+
+        // Check volume emptiness, again
+        if (!omMetadataManager.isVolumeEmpty(volumeName)) {
+          LOG.debug("volume: '{}' is not empty", volumeName);
+          throw new OMException("Aborting tenant deletion. " +
+              "Volume becomes non-empty somewhere between" +
+              "preExecute and validateAndUpdateCache", VOLUME_NOT_EMPTY);
+        }
+
+        // Actual volume deletion, follows OMVolumeDeleteRequest
+        newVolumeList =
+            omMetadataManager.getUserTable().get(volumeOwner);
+        newVolumeList = delVolumeFromOwnerList(newVolumeList, volumeName,
+            volumeOwner, transactionLogIndex);
+        final String dbUserKey = omMetadataManager.getUserKey(volumeOwner);
+        omMetadataManager.getUserTable().addCacheEntry(
+            new CacheKey<>(dbUserKey),
+            new CacheValue<>(Optional.of(newVolumeList), transactionLogIndex));
+        final String dbVolumeKey = omMetadataManager.getVolumeKey(volumeName);
+        omMetadataManager.getVolumeTable().addCacheEntry(
+            new CacheKey<>(dbVolumeKey),
+            new CacheValue<>(Optional.absent(), transactionLogIndex));
+
+        // TODO: Set response dbVolumeKey?
+      }
+
+      // TODO: Should hold some tenant lock here. Just in case !deleteVolume
+//      acquiredTenantLock = omMetadataManager.getLock().acquireWriteLock(
+//          TENANT_LOCK, tenantId);

Review comment:
       What would the lock be for? If we need it we should add it now or we'll forget about it.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/VolumeArgs.java
##########
@@ -115,6 +117,10 @@ public long getQuotaInNamespace() {
     return new VolumeArgs.Builder();
   }
 
+  public long getRefCount() {

Review comment:
       This class is used by the client to create a new volume. Probably no need to have ref count here.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // TODO: Check tenantId validity? Maybe not
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    // Check if the tenant volume is empty (when the volume exists)
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getAssociatedVolumeName();
+    // Note: Looks like isVolumeEmpty() returns true if volume doesn't exist
+    if (volumeName.length() > 0 && !metadataManager.isVolumeEmpty(volumeName)) {
+      LOG.debug("volume: {} is not empty", volumeName);
+      throw new OMException("Tenant volume '" + volumeName + "' is not empty." +
+          " Volume must be emptied before the tenant can be deleted.",
+          VOLUME_NOT_EMPTY);
+    }
+
+    // Check if there are any accessIds in the tenant
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be ignored?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  public void handleRequestFailure(OzoneManager ozoneManager) {
+    super.handleRequestFailure(ozoneManager);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false, acquiredUserLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string
+    final String volumeName =
+        getOmRequest().getDeleteVolumeRequest().getVolumeName();
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs;
+    String volumeOwner = null;
+    // deleteVolume is true if volumeName is not empty string
+    boolean deleteVolume = volumeName.length() > 0;
+    OzoneManagerStorageProtos.PersistedUserVolumeInfo newVolumeList = null;
+
+    try {
+      if (deleteVolume) {
+        // check Acl
+        if (ozoneManager.getAclsEnabled()) {
+          checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+              OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+              volumeName, null, null);
+        }
+
+        acquiredVolumeLock = omMetadataManager.getLock().acquireWriteLock(
+            VOLUME_LOCK, volumeName);
+
+        omVolumeArgs = getVolumeInfo(omMetadataManager, volumeName);
+        // Check volume ref count
+        long volRefCount = omVolumeArgs.getRefCount();
+        if (volRefCount > 1L) {
+          LOG.warn("Volume '{}' has a greater than 1 reference count of " +
+                  "'{}'", volumeName, volRefCount);
+          throw new OMException("Volume '" + volumeName + "' has a greater " +
+              "than 1 reference count of '" + volRefCount + "'. This may " +
+              "indicate the volume is in use by some other Ozone features. " +
+              "Please disable such other features before trying to delete " +
+              "the tenant again.", VOLUME_IN_USE);
+        }
+
+        volumeOwner = omVolumeArgs.getOwnerName();
+        acquiredUserLock = omMetadataManager.getLock().acquireWriteLock(
+            USER_LOCK, volumeOwner);
+
+        // Check volume emptiness, again
+        if (!omMetadataManager.isVolumeEmpty(volumeName)) {
+          LOG.debug("volume: '{}' is not empty", volumeName);
+          throw new OMException("Aborting tenant deletion. " +
+              "Volume becomes non-empty somewhere between" +
+              "preExecute and validateAndUpdateCache", VOLUME_NOT_EMPTY);

Review comment:
       Can we make this more user friendly? User does not need to know about our method names, that's for log messages.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // TODO: Check tenantId validity? Maybe not
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    // Check if the tenant volume is empty (when the volume exists)
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getAssociatedVolumeName();
+    // Note: Looks like isVolumeEmpty() returns true if volume doesn't exist
+    if (volumeName.length() > 0 && !metadataManager.isVolumeEmpty(volumeName)) {
+      LOG.debug("volume: {} is not empty", volumeName);
+      throw new OMException("Tenant volume '" + volumeName + "' is not empty." +
+          " Volume must be emptied before the tenant can be deleted.",
+          VOLUME_NOT_EMPTY);
+    }
+
+    // Check if there are any accessIds in the tenant
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be ignored?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  public void handleRequestFailure(OzoneManager ozoneManager) {
+    super.handleRequestFailure(ozoneManager);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false, acquiredUserLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string
+    final String volumeName =
+        getOmRequest().getDeleteVolumeRequest().getVolumeName();
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs;
+    String volumeOwner = null;
+    // deleteVolume is true if volumeName is not empty string
+    boolean deleteVolume = volumeName.length() > 0;
+    OzoneManagerStorageProtos.PersistedUserVolumeInfo newVolumeList = null;
+
+    try {
+      if (deleteVolume) {
+        // check Acl
+        if (ozoneManager.getAclsEnabled()) {
+          checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+              OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+              volumeName, null, null);
+        }
+
+        acquiredVolumeLock = omMetadataManager.getLock().acquireWriteLock(
+            VOLUME_LOCK, volumeName);
+
+        omVolumeArgs = getVolumeInfo(omMetadataManager, volumeName);
+        // Check volume ref count
+        long volRefCount = omVolumeArgs.getRefCount();
+        if (volRefCount > 1L) {

Review comment:
       This > check is why this request did not catch that this value is never persisted. It is never initialized and defaults to 0. Actually if the value is 0 here we have a problem and should at least log a warning.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmVolumeArgs.java
##########
@@ -48,6 +48,19 @@
   private long quotaInNamespace;
   private long usedNamespace;
   private List<OzoneAcl> acls;
+  /**
+   * Reference count on this Ozone volume.
+   *
+   * When reference count is larger than zero, it indicates that at least one
+   * "lock" is held on the volume by some Ozone feature (e.g. multi-tenancy).
+   * Volume delete operation will be denied in this case, and user should be
+   * prompted to release the lock first via the interface provided by that
+   * feature.
+   *
+   * Volumes created using CLI, ObjectStore API or upgraded from older OM DB
+   * will have reference count set to zero by default.
+   */
+  private long refCount;

Review comment:
       This needs to be added to `getFromProtobuf`, `getProtobuf`, and `VolumeInfo` proto. If the value is not present, we can just set it to 0. Right now this does nothing, which indicates we need to add testing for existing volume create/delete requests and delete of a tenant volume, which would have caught this.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantRequestHelper.java
##########
@@ -94,26 +99,26 @@ static void checkTenantExistence(OMMetadataManager omMetadataManager,
    * Retrieve volume name of the tenant.
    */
   static String getTenantVolumeName(OMMetadataManager omMetadataManager,
-      String tenantName) {
+      String tenantId) {
 
     final OmDBTenantInfo tenantInfo;
     try {
-      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantName);
+      tenantInfo = omMetadataManager.getTenantStateTable().get(tenantId);
     } catch (IOException e) {
       throw new RuntimeException("Potential DB error. Unable to retrieve "
-          + "OmDBTenantInfo entry for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry for tenant '" + tenantId + "'.");
     }
 
     if (tenantInfo == null) {
       throw new RuntimeException("Potential DB error or race condition. "
-          + "OmDBTenantInfo entry is missing for tenant '" + tenantName + "'.");
+          + "OmDBTenantInfo entry is missing for tenant '" + tenantId + "'.");
     }
 
-    final String volumeName = tenantInfo.getAccountNamespaceName();
+    final String volumeName = tenantInfo.getBucketNamespaceName();
 
-    if (StringUtils.isEmpty(tenantName)) {
+    if (StringUtils.isEmpty(tenantId)) {

Review comment:
       Done. but throwing `METADATA_ERROR` instead. The `volumeName` field is not supposed to be null (but could be "")




-- 
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] smengcl merged pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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


   


-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // TODO: Check tenantId validity? Maybe not
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    // Check if the tenant volume is empty (when the volume exists)
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getAssociatedVolumeName();
+    // Note: Looks like isVolumeEmpty() returns true if volume doesn't exist
+    if (volumeName.length() > 0 && !metadataManager.isVolumeEmpty(volumeName)) {
+      LOG.debug("volume: {} is not empty", volumeName);
+      throw new OMException("Tenant volume '" + volumeName + "' is not empty." +
+          " Volume must be emptied before the tenant can be deleted.",
+          VOLUME_NOT_EMPTY);
+    }
+
+    // Check if there are any accessIds in the tenant
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be ignored?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  public void handleRequestFailure(OzoneManager ozoneManager) {
+    super.handleRequestFailure(ozoneManager);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false, acquiredUserLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string
+    final String volumeName =
+        getOmRequest().getDeleteVolumeRequest().getVolumeName();
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs;
+    String volumeOwner = null;
+    // deleteVolume is true if volumeName is not empty string
+    boolean deleteVolume = volumeName.length() > 0;
+    OzoneManagerStorageProtos.PersistedUserVolumeInfo newVolumeList = null;
+
+    try {
+      if (deleteVolume) {
+        // check Acl
+        if (ozoneManager.getAclsEnabled()) {
+          checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+              OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+              volumeName, null, null);
+        }
+
+        acquiredVolumeLock = omMetadataManager.getLock().acquireWriteLock(
+            VOLUME_LOCK, volumeName);
+
+        omVolumeArgs = getVolumeInfo(omMetadataManager, volumeName);
+        // Check volume ref count
+        long volRefCount = omVolumeArgs.getRefCount();
+        if (volRefCount > 1L) {
+          LOG.warn("Volume '{}' has a greater than 1 reference count of " +
+                  "'{}'", volumeName, volRefCount);
+          throw new OMException("Volume '" + volumeName + "' has a greater " +
+              "than 1 reference count of '" + volRefCount + "'. This may " +
+              "indicate the volume is in use by some other Ozone features. " +
+              "Please disable such other features before trying to delete " +
+              "the tenant again.", VOLUME_IN_USE);
+        }
+
+        volumeOwner = omVolumeArgs.getOwnerName();
+        acquiredUserLock = omMetadataManager.getLock().acquireWriteLock(
+            USER_LOCK, volumeOwner);
+
+        // Check volume emptiness, again
+        if (!omMetadataManager.isVolumeEmpty(volumeName)) {
+          LOG.debug("volume: '{}' is not empty", volumeName);
+          throw new OMException("Aborting tenant deletion. " +
+              "Volume becomes non-empty somewhere between" +
+              "preExecute and validateAndUpdateCache", VOLUME_NOT_EMPTY);
+        }
+
+        // Actual volume deletion, follows OMVolumeDeleteRequest
+        newVolumeList =
+            omMetadataManager.getUserTable().get(volumeOwner);
+        newVolumeList = delVolumeFromOwnerList(newVolumeList, volumeName,
+            volumeOwner, transactionLogIndex);
+        final String dbUserKey = omMetadataManager.getUserKey(volumeOwner);
+        omMetadataManager.getUserTable().addCacheEntry(
+            new CacheKey<>(dbUserKey),
+            new CacheValue<>(Optional.of(newVolumeList), transactionLogIndex));
+        final String dbVolumeKey = omMetadataManager.getVolumeKey(volumeName);
+        omMetadataManager.getVolumeTable().addCacheEntry(
+            new CacheKey<>(dbVolumeKey),
+            new CacheValue<>(Optional.absent(), transactionLogIndex));
+
+        // TODO: Set response dbVolumeKey?
+      }
+
+      // TODO: Should hold some tenant lock here. Just in case !deleteVolume
+//      acquiredTenantLock = omMetadataManager.getLock().acquireWriteLock(
+//          TENANT_LOCK, tenantId);

Review comment:
       The discussion concluded that `VOLUME_LOCK` would be used to protect tenant tables, with the exception that reading from `TenantStateTable` (or cache) will not be protected by any locks because entries in `TenantStateTable` won't be updated once it is written to by `CreateTenant` (i.e. no `UpdateTenant`).




-- 
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] errose28 commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/tenant/OMTenantDeleteRequest.java
##########
@@ -18,33 +18,260 @@
  */
 package org.apache.hadoop.ozone.om.request.s3.tenant;
 
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMultiTenantManager;
 import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OmDBTenantInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
 import org.apache.hadoop.ozone.om.request.volume.OMVolumeRequest;
 import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.s3.tenant.OMTenantDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteTenantRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.ozone.storage.proto.OzoneManagerStorageProtos;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TENANT_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_IN_USE;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.USER_LOCK;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
 
 /**
  * Handles OMTenantDelete request.
  */
 public class OMTenantDeleteRequest extends OMVolumeRequest {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(OMTenantDeleteRequest.class);
 
   public OMTenantDeleteRequest(OMRequest omRequest) {
     super(omRequest);
   }
 
   @Override
   public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
-    return getOmRequest();
+
+    // Check Ozone cluster admin privilege
+    OMTenantRequestHelper.checkAdmin(ozoneManager);
+
+    final OMMetadataManager metadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+
+    // TODO: Check tenantId validity? Maybe not
+
+    // Check tenant existence in tenantStateTable
+    if (!metadataManager.getTenantStateTable().isExist(tenantId)) {
+      LOG.debug("tenant: {} does not exist", tenantId);
+      throw new OMException("Tenant '" + tenantId + "' does not exist",
+          TENANT_NOT_FOUND);
+    }
+
+    // Check if the tenant volume is empty (when the volume exists)
+    final OmDBTenantInfo dbTenantInfo =
+        metadataManager.getTenantStateTable().get(tenantId);
+    final String volumeName = dbTenantInfo.getAssociatedVolumeName();
+    // Note: Looks like isVolumeEmpty() returns true if volume doesn't exist
+    if (volumeName.length() > 0 && !metadataManager.isVolumeEmpty(volumeName)) {
+      LOG.debug("volume: {} is not empty", volumeName);
+      throw new OMException("Tenant volume '" + volumeName + "' is not empty." +
+          " Volume must be emptied before the tenant can be deleted.",
+          VOLUME_NOT_EMPTY);
+    }
+
+    // Check if there are any accessIds in the tenant
+    final OMMultiTenantManager tenantManager =
+        ozoneManager.getMultiTenantManager();
+    if (!OMTenantRequestHelper.isTenantEmpty(tenantManager, tenantId)) {
+      LOG.warn("tenant: '{}' is not empty. Unable to delete the tenant",
+          tenantId);
+      throw new OMException("Tenant '" + tenantId + "' is not empty. " +
+          "All accessIds associated to this tenant must be revoked before " +
+          "the tenant can be deleted.", TENANT_NOT_EMPTY);
+    }
+
+    // TODO: TBD: Call ozoneManager.getMultiTenantManager().deleteTenant() ?
+
+    // Regenerate request with the volumeName
+    final OMRequest.Builder omRequestBuilder = getOmRequest().toBuilder()
+        .setDeleteTenantRequest(DeleteTenantRequest.newBuilder()
+            .setTenantId(tenantId))
+        .setDeleteVolumeRequest(DeleteVolumeRequest.newBuilder()
+            .setVolumeName(volumeName))
+        // TODO: Can the three lines below be ignored?
+        .setUserInfo(getUserInfo())
+        .setCmdType(getOmRequest().getCmdType())
+        .setClientId(getOmRequest().getClientId());
+
+    return omRequestBuilder.build();
   }
 
   @Override
+  public void handleRequestFailure(OzoneManager ozoneManager) {
+    super.handleRequestFailure(ozoneManager);
+  }
+
+  @Override
+  @SuppressWarnings("methodlength")
   public OMClientResponse validateAndUpdateCache(
       OzoneManager ozoneManager, long transactionLogIndex,
       OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
 
-    return null;
+    OMClientResponse omClientResponse = null;
+    final OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    boolean acquiredVolumeLock = false, acquiredUserLock = false;
+    final Map<String, String> auditMap = new HashMap<>();
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+    final DeleteTenantRequest request = getOmRequest().getDeleteTenantRequest();
+    final String tenantId = request.getTenantId();
+    // NOTE: volumeName might be a zero-length string
+    final String volumeName =
+        getOmRequest().getDeleteVolumeRequest().getVolumeName();
+
+    IOException exception = null;
+    OmVolumeArgs omVolumeArgs;
+    String volumeOwner = null;
+    // deleteVolume is true if volumeName is not empty string
+    boolean deleteVolume = volumeName.length() > 0;
+    OzoneManagerStorageProtos.PersistedUserVolumeInfo newVolumeList = null;
+
+    try {
+      if (deleteVolume) {
+        // check Acl
+        if (ozoneManager.getAclsEnabled()) {
+          checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
+              OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
+              volumeName, null, null);
+        }
+
+        acquiredVolumeLock = omMetadataManager.getLock().acquireWriteLock(
+            VOLUME_LOCK, volumeName);
+
+        omVolumeArgs = getVolumeInfo(omMetadataManager, volumeName);
+        // Check volume ref count
+        long volRefCount = omVolumeArgs.getRefCount();
+        if (volRefCount > 1L) {
+          LOG.warn("Volume '{}' has a greater than 1 reference count of " +
+                  "'{}'", volumeName, volRefCount);
+          throw new OMException("Volume '" + volumeName + "' has a greater " +
+              "than 1 reference count of '" + volRefCount + "'. This may " +
+              "indicate the volume is in use by some other Ozone features. " +
+              "Please disable such other features before trying to delete " +
+              "the tenant again.", VOLUME_IN_USE);
+        }
+
+        volumeOwner = omVolumeArgs.getOwnerName();
+        acquiredUserLock = omMetadataManager.getLock().acquireWriteLock(
+            USER_LOCK, volumeOwner);
+
+        // Check volume emptiness, again
+        if (!omMetadataManager.isVolumeEmpty(volumeName)) {
+          LOG.debug("volume: '{}' is not empty", volumeName);
+          throw new OMException("Aborting tenant deletion. " +
+              "Volume becomes non-empty somewhere between" +
+              "preExecute and validateAndUpdateCache", VOLUME_NOT_EMPTY);
+        }
+
+        // Actual volume deletion, follows OMVolumeDeleteRequest
+        newVolumeList =
+            omMetadataManager.getUserTable().get(volumeOwner);
+        newVolumeList = delVolumeFromOwnerList(newVolumeList, volumeName,
+            volumeOwner, transactionLogIndex);
+        final String dbUserKey = omMetadataManager.getUserKey(volumeOwner);
+        omMetadataManager.getUserTable().addCacheEntry(
+            new CacheKey<>(dbUserKey),
+            new CacheValue<>(Optional.of(newVolumeList), transactionLogIndex));
+        final String dbVolumeKey = omMetadataManager.getVolumeKey(volumeName);
+        omMetadataManager.getVolumeTable().addCacheEntry(
+            new CacheKey<>(dbVolumeKey),
+            new CacheValue<>(Optional.absent(), transactionLogIndex));
+
+        // TODO: Set response dbVolumeKey?
+      }
+
+      // TODO: Should hold some tenant lock here. Just in case !deleteVolume
+//      acquiredTenantLock = omMetadataManager.getLock().acquireWriteLock(
+//          TENANT_LOCK, tenantId);

Review comment:
       Ok, lets discuss offline and then we can update here with the result.




-- 
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] smengcl commented on a change in pull request #2857: HDDS-6022. [Multi-Tenant] Implement DeleteTenant: `ozone tenant delete`

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



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/tenant/TenantCreateHandler.java
##########
@@ -33,26 +32,19 @@
     description = "Create one or more tenants")
 public class TenantCreateHandler extends TenantHandler {
 
-  @CommandLine.Spec
-  private CommandLine.Model.CommandSpec spec;
-
-  @CommandLine.Parameters(description = "List of tenant names")
+  @CommandLine.Parameters(description = "List of tenant names", arity = "1..")
   private List<String> tenants = new ArrayList<>();
 
   @Override
   protected void execute(OzoneClient client, OzoneAddress address) {
-    if (tenants.size() > 0) {
-      for (String tenantName : tenants) {
-        try {
-          client.getObjectStore().createTenant(tenantName);
-          out().println("Created tenant '" + tenantName + "'.");
-        } catch (IOException e) {
-          err().println("Failed to create tenant '" + tenantName + "': " +
-              e.getMessage());
-        }
+    for (String tenantId : tenants) {

Review comment:
       I think I will file a jira to remove the loop for create/delete tenant CLI.
   
   But this might actually have value for assign user to tenant? i.e. An admin batch assigning users to a tenant without waiting for seconds every time the Ozone CLI client's JVM boots up.




-- 
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