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 2023/01/13 04:15:44 UTC

[GitHub] [ozone] smengcl opened a new pull request, #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

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

   ## What changes were proposed in this pull request?
   
   - Implement CLI `ozone sh snapshot delete`.
   - Implement Ratis Tx `OMSnapshotDeleteRequest`.
   - Improve CLI output for UX.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-6857
   
   ## How was this patch tested?
   
   - [x] Add UT `TestOMSnapshotDeleteRequest`.
   - [ ] TODO: Finish UT `TestOMSnapshotDeleteResponse`.
   - [x] Tested manually in Docker locally:
   
   ```bash
   $ ozone fs -mkdir -p ofs://om/vol1/buck1/dir1/
   $ ozone fs -put README.md ofs://om/vol1/buck1/dir1/
   $ ozone sh snapshot create /vol1/buck1 snap1
   $ ozone sh snapshot delete /vol1/buck1 snap1
   $ ozone sh snapshot delete /vol1/buck1 snap1
   FILE_NOT_FOUND Snapshot exists but no longer active
   $ ozone sh snapshot list /vol1/buck1
   [ {
     "volumeName" : "vol1",
     "bucketName" : "buck1",
     "name" : "snap1",
     "creationTime" : 1673583279047,
     "snapshotStatus" : "SNAPSHOT_DELETED",
     "snapshotID" : "08f640e3-95db-4596-9d5f-a47cb7272329",
     "snapshotPath" : "vol1/buck1",
     "checkpointDir" : "-08f640e3-95db-4596-9d5f-a47cb7272329"
   } ]
   ```


-- 
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 diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1092374753


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the

Review Comment:
   Hi Hemant, `DeleteSnapshotRequest` does not depend on snapshotId with the current approach.
   
   Regarding `omRequest`, we are building on top of the incoming request with `omRequest.toBuilder()` and rewriting `DeleteSnapshotRequest` params only. Other request params should have been retained as-is (e.g. `UserInfo`).



-- 
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 diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1088259915


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the
+    // exact same timestamp.
+    OMRequest.Builder omRequestBuilder = omRequest.toBuilder()
+        .setDeleteSnapshotRequest(
+            DeleteSnapshotRequest.newBuilder()
+                .setVolumeName(volumeName)
+                .setBucketName(bucketName)
+                .setSnapshotName(snapshotName)
+                .setDeletionTime(Time.now()));
+
+    return omRequestBuilder.build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumSnapshotDeletes();
+
+//    boolean acquiredBucketLock = false;
+    boolean acquiredSnapshotLock = false;
+    IOException exception = null;
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+        getOmRequest());
+    OMClientResponse omClientResponse = null;
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    UserInfo userInfo = getOmRequest().getUserInfo();
+
+    final DeleteSnapshotRequest request =
+        getOmRequest().getDeleteSnapshotRequest();
+
+    final String volumeName = request.getVolumeName();
+    final String bucketName = request.getBucketName();
+    final String snapshotName = request.getSnapshotName();
+    final long deletionTime = request.getDeletionTime();
+
+    SnapshotInfo snapshotInfo = null;
+
+    try {
+      // TODO: Do we need a bucket lock here? Probably not?
+//      acquiredBucketLock =
+//          omMetadataManager.getLock().acquireReadLock(BUCKET_LOCK,
+//              volumeName, bucketName);
+
+      acquiredSnapshotLock =
+          omMetadataManager.getLock().acquireWriteLock(SNAPSHOT_LOCK,
+              volumeName, bucketName, snapshotName);
+
+      // Retrieve SnapshotInfo from the table
+      String tableKey = SnapshotInfo.getTableKey(volumeName, bucketName,
+          snapshotName);
+      snapshotInfo =
+          omMetadataManager.getSnapshotInfoTable().get(tableKey);
+
+      if (snapshotInfo == null) {

Review Comment:
   The previous check is done in `preExecute`. Thus, we have to check it again here in `validateAndUpdateCache` because different stages from multiple requests can interleave.



-- 
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 diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1092369381


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the
+    // exact same timestamp.
+    OMRequest.Builder omRequestBuilder = omRequest.toBuilder()
+        .setDeleteSnapshotRequest(
+            DeleteSnapshotRequest.newBuilder()
+                .setVolumeName(volumeName)
+                .setBucketName(bucketName)
+                .setSnapshotName(snapshotName)
+                .setDeletionTime(Time.now()));
+
+    return omRequestBuilder.build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumSnapshotDeletes();
+
+//    boolean acquiredBucketLock = false;
+    boolean acquiredSnapshotLock = false;
+    IOException exception = null;
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+        getOmRequest());
+    OMClientResponse omClientResponse = null;
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    UserInfo userInfo = getOmRequest().getUserInfo();
+
+    final DeleteSnapshotRequest request =
+        getOmRequest().getDeleteSnapshotRequest();
+
+    final String volumeName = request.getVolumeName();
+    final String bucketName = request.getBucketName();
+    final String snapshotName = request.getSnapshotName();
+    final long deletionTime = request.getDeletionTime();
+
+    SnapshotInfo snapshotInfo = null;
+
+    try {
+      // TODO: Do we need a bucket lock here? Probably not?
+//      acquiredBucketLock =
+//          omMetadataManager.getLock().acquireReadLock(BUCKET_LOCK,
+//              volumeName, bucketName);
+
+      acquiredSnapshotLock =
+          omMetadataManager.getLock().acquireWriteLock(SNAPSHOT_LOCK,
+              volumeName, bucketName, snapshotName);
+
+      // Retrieve SnapshotInfo from the table
+      String tableKey = SnapshotInfo.getTableKey(volumeName, bucketName,
+          snapshotName);
+      snapshotInfo =
+          omMetadataManager.getSnapshotInfoTable().get(tableKey);
+
+      if (snapshotInfo == null) {
+        // Snapshot does not exist
+        throw new OMException("Snapshot does not exist", FILE_NOT_FOUND);
+      }
+
+      if (!snapshotInfo.getSnapshotStatus().equals(
+          SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE)) {
+        // If the snapshot is not in active state, throw exception as well
+        throw new OMException("Snapshot exists but no longer active",

Review Comment:
   "Snapshot pending deletion" would only apply to `SNAPSHOT_DELETED`.
   
   I have detailed the snapshot status a bit. Please take another look.



-- 
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 #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on PR #4175:
URL: https://github.com/apache/ozone/pull/4175#issuecomment-1405625067

   > With the snapshot delete cli command is it possible to indicate to the user the amount of space that will be freed with the operation? If not here, then through another method.
   
   Hi Neil, we do have the snapshot disk usage in consideration in HDDS-7744 and has been detailed in the design doc.
   
   > ```
   > $ ozone sh snapshot list /vol1/buck1
   > [ {
   >   "volumeName" : "vol1",
   >   "bucketName" : "buck1",
   >   "name" : "snap1",
   >   "creationTime" : 1673583279047,
   >   "snapshotStatus" : "SNAPSHOT_DELETED",
   >   "snapshotID" : "08f640e3-95db-4596-9d5f-a47cb7272329",
   >   "snapshotPath" : "vol1/buck1",
   >   "checkpointDir" : "-08f640e3-95db-4596-9d5f-a47cb7272329"
   > } ]
   > ```
   > 
   > In the example provided, after executing the delete, listing the snapshots (shoiwn above) for the bucket shows the "SNAPSHOT_DELETED" state. With the current example it appears that listing snapshots for the bucket always shows a snapshot in deleted state. We expect it to be removed from the listing at some point? After garbage collection?
   
   It will be cleaned up and removed by `SnapshotDeletingTask` later, being implemented in HDDS-7740.


-- 
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 diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1092370690


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1678,6 +1681,13 @@ message SnapshotDiffRequest {
   required string toSnapshot = 4;
 }
 
+message DeleteSnapshotRequest {
+  optional string volumeName = 1;
+  optional string bucketName = 2;
+  optional string snapshotName = 3;
+  optional uint64 deletionTime = 4;

Review Comment:
   `deletionTime` is here to be synced across OMs. Otherwise the deletionTime in the SnapshotInfo entry for a snapshot won't be guaranteed synced if each OM generates the timestamp themselves.



-- 
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] hemantk-12 commented on a diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1092489550


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the

Review Comment:
   I meant we are setting deletion time at line 100. According to me, this timestamp might not be the same for leader and follower if it matters.
   
   Sorry if I was not clear in the previous comment.



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "aswinshakil (via GitHub)" <gi...@apache.org>.
aswinshakil commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1087454672


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the
+    // exact same timestamp.
+    OMRequest.Builder omRequestBuilder = omRequest.toBuilder()
+        .setDeleteSnapshotRequest(
+            DeleteSnapshotRequest.newBuilder()
+                .setVolumeName(volumeName)
+                .setBucketName(bucketName)
+                .setSnapshotName(snapshotName)
+                .setDeletionTime(Time.now()));
+
+    return omRequestBuilder.build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumSnapshotDeletes();
+
+//    boolean acquiredBucketLock = false;
+    boolean acquiredSnapshotLock = false;
+    IOException exception = null;
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+        getOmRequest());
+    OMClientResponse omClientResponse = null;
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    UserInfo userInfo = getOmRequest().getUserInfo();
+
+    final DeleteSnapshotRequest request =
+        getOmRequest().getDeleteSnapshotRequest();
+
+    final String volumeName = request.getVolumeName();
+    final String bucketName = request.getBucketName();
+    final String snapshotName = request.getSnapshotName();
+    final long deletionTime = request.getDeletionTime();
+
+    SnapshotInfo snapshotInfo = null;
+
+    try {
+      // TODO: Do we need a bucket lock here? Probably not?
+//      acquiredBucketLock =
+//          omMetadataManager.getLock().acquireReadLock(BUCKET_LOCK,
+//              volumeName, bucketName);
+
+      acquiredSnapshotLock =
+          omMetadataManager.getLock().acquireWriteLock(SNAPSHOT_LOCK,
+              volumeName, bucketName, snapshotName);
+
+      // Retrieve SnapshotInfo from the table
+      String tableKey = SnapshotInfo.getTableKey(volumeName, bucketName,
+          snapshotName);
+      snapshotInfo =
+          omMetadataManager.getSnapshotInfoTable().get(tableKey);
+
+      if (snapshotInfo == null) {

Review Comment:
   We already do a check here, Essentially this would be `null` if the `snapshotName` is invalid. Is it necessary to validate the snapshot name at multiple places? `OmUtils.validateSnapshotName()`



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/snapshot/TestOMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,331 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import com.google.common.base.Optional;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.db.cache.CacheKey;
+import org.apache.hadoop.hdds.utils.db.cache.CacheValue;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.AuditMessage;
+import org.apache.hadoop.ozone.om.OMConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OmMetadataManagerImpl;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ozone.test.LambdaTestUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.mockito.Mockito;
+
+import java.util.UUID;
+
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests OMSnapshotDeleteRequest class, which handles DeleteSnapshot request.
+ * Mostly mirrors TestOMSnapshotCreateRequest.
+ * testEntryNotExist() and testEntryExists() are unique.
+ */
+public class TestOMSnapshotDeleteRequest {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+
+  private OzoneManager ozoneManager;
+  private OMMetrics omMetrics;
+  private OMMetadataManager omMetadataManager;
+
+  private String volumeName;
+  private String bucketName;
+  private String snapshotName;
+
+  // Just setting ozoneManagerDoubleBuffer which does nothing.
+  private final OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper =
+      ((response, transactionIndex) -> null);
+
+  @Before
+  public void setup() throws Exception {
+
+    ozoneManager = mock(OzoneManager.class);
+    omMetrics = OMMetrics.create();
+    OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
+    ozoneConfiguration.set(OMConfigKeys.OZONE_OM_DB_DIRS,
+        folder.newFolder().getAbsolutePath());
+    omMetadataManager = new OmMetadataManagerImpl(ozoneConfiguration);
+    when(ozoneManager.getMetrics()).thenReturn(omMetrics);
+    when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager);
+    when(ozoneManager.isRatisEnabled()).thenReturn(true);
+    when(ozoneManager.isAdmin(any())).thenReturn(false);
+    when(ozoneManager.isOwner(any(), any())).thenReturn(false);
+    when(ozoneManager.getBucketOwner(any(), any(),
+        any(), any())).thenReturn("dummyBucketOwner");
+    OMLayoutVersionManager lvm = mock(OMLayoutVersionManager.class);
+    when(lvm.getMetadataLayoutVersion()).thenReturn(0);
+    when(ozoneManager.getVersionManager()).thenReturn(lvm);
+    AuditLogger auditLogger = mock(AuditLogger.class);
+    when(ozoneManager.getAuditLogger()).thenReturn(auditLogger);
+    Mockito.doNothing().when(auditLogger).logWrite(any(AuditMessage.class));
+
+    volumeName = UUID.randomUUID().toString();
+    bucketName = UUID.randomUUID().toString();
+    snapshotName = UUID.randomUUID().toString();
+    OMRequestTestUtils.addVolumeAndBucketToDB(
+        volumeName, bucketName, omMetadataManager);
+
+  }
+
+  @After
+  public void stop() {
+    omMetrics.unRegister();
+    Mockito.framework().clearInlineMocks();
+  }
+
+  @Test
+  public void testPreExecute() throws Exception {
+    // set the owner
+    when(ozoneManager.isOwner(any(), any())).thenReturn(true);
+    OMRequest omRequest =
+        OMRequestTestUtils.deleteSnapshotRequest(
+        volumeName, bucketName, snapshotName);
+    // should not throw
+    doPreExecute(omRequest);
+  }
+
+  @Test
+  public void testPreExecuteBadOwner() throws Exception {
+    // owner not set
+    OMRequest omRequest =
+        OMRequestTestUtils.deleteSnapshotRequest(
+        volumeName, bucketName, snapshotName);
+    // Check bad owner
+    LambdaTestUtils.intercept(OMException.class,
+        "Only bucket owners and Ozone admins can delete snapshots",
+        () -> doPreExecute(omRequest));
+  }
+
+  @Test
+  public void testPreExecuteBadName() throws Exception {

Review Comment:
   These tests can be removed if we decide to remove `OmUtils.validateSnapshotName()` in `preExecute`



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

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

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


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


[GitHub] [ozone] prashantpogde commented on a diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "prashantpogde (via GitHub)" <gi...@apache.org>.
prashantpogde commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1087185529


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the
+    // exact same timestamp.
+    OMRequest.Builder omRequestBuilder = omRequest.toBuilder()
+        .setDeleteSnapshotRequest(
+            DeleteSnapshotRequest.newBuilder()
+                .setVolumeName(volumeName)
+                .setBucketName(bucketName)
+                .setSnapshotName(snapshotName)
+                .setDeletionTime(Time.now()));
+
+    return omRequestBuilder.build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumSnapshotDeletes();
+
+//    boolean acquiredBucketLock = false;
+    boolean acquiredSnapshotLock = false;
+    IOException exception = null;
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+        getOmRequest());
+    OMClientResponse omClientResponse = null;
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    UserInfo userInfo = getOmRequest().getUserInfo();
+
+    final DeleteSnapshotRequest request =
+        getOmRequest().getDeleteSnapshotRequest();
+
+    final String volumeName = request.getVolumeName();
+    final String bucketName = request.getBucketName();
+    final String snapshotName = request.getSnapshotName();
+    final long deletionTime = request.getDeletionTime();
+
+    SnapshotInfo snapshotInfo = null;
+
+    try {
+      // TODO: Do we need a bucket lock here? Probably not?

Review Comment:
   Bucket lock is needed 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 pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on PR #4175:
URL: https://github.com/apache/ozone/pull/4175#issuecomment-1411476340

   Thanks @prashantpogde for the +1. CI passed. I will merge this to unblock other GC work.
   
   I've filed HDDS-7862 for the `TestOMSnapshotDeleteResponse` UT TODO.
   
   Thanks @hemantk-12 @aswinshakil @neils-dev for the reviews.


-- 
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] neils-dev commented on pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by GitBox <gi...@apache.org>.
neils-dev commented on PR #4175:
URL: https://github.com/apache/ozone/pull/4175#issuecomment-1396472566

   With the snapshot delete cli command is it possible to indicate to the user the amount of space that will be freed with the operation?  If not here, then through another method.
   
   ```
   $ ozone sh snapshot list /vol1/buck1
   [ {
     "volumeName" : "vol1",
     "bucketName" : "buck1",
     "name" : "snap1",
     "creationTime" : 1673583279047,
     "snapshotStatus" : "SNAPSHOT_DELETED",
     "snapshotID" : "08f640e3-95db-4596-9d5f-a47cb7272329",
     "snapshotPath" : "vol1/buck1",
     "checkpointDir" : "-08f640e3-95db-4596-9d5f-a47cb7272329"
   } ]
   ```
   
   In the example provided, after executing the delete, listing the snapshots (shoiwn above) for the bucket shows the "SNAPSHOT_DELETED" state.  With the current example it appears that listing snapshots for the bucket always shows a snapshot in deleted state.  We expect it to be removed from the listing at some point?  After garbage collection? 


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

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

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


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


[GitHub] [ozone] prashantpogde commented on a diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "prashantpogde (via GitHub)" <gi...@apache.org>.
prashantpogde commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1087187958


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the
+    // exact same timestamp.
+    OMRequest.Builder omRequestBuilder = omRequest.toBuilder()
+        .setDeleteSnapshotRequest(
+            DeleteSnapshotRequest.newBuilder()
+                .setVolumeName(volumeName)
+                .setBucketName(bucketName)
+                .setSnapshotName(snapshotName)
+                .setDeletionTime(Time.now()));
+
+    return omRequestBuilder.build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumSnapshotDeletes();
+
+//    boolean acquiredBucketLock = false;
+    boolean acquiredSnapshotLock = false;
+    IOException exception = null;
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+        getOmRequest());
+    OMClientResponse omClientResponse = null;
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    UserInfo userInfo = getOmRequest().getUserInfo();
+
+    final DeleteSnapshotRequest request =
+        getOmRequest().getDeleteSnapshotRequest();
+
+    final String volumeName = request.getVolumeName();
+    final String bucketName = request.getBucketName();
+    final String snapshotName = request.getSnapshotName();
+    final long deletionTime = request.getDeletionTime();
+
+    SnapshotInfo snapshotInfo = null;
+
+    try {
+      // TODO: Do we need a bucket lock here? Probably not?
+//      acquiredBucketLock =
+//          omMetadataManager.getLock().acquireReadLock(BUCKET_LOCK,
+//              volumeName, bucketName);
+
+      acquiredSnapshotLock =
+          omMetadataManager.getLock().acquireWriteLock(SNAPSHOT_LOCK,
+              volumeName, bucketName, snapshotName);
+
+      // Retrieve SnapshotInfo from the table
+      String tableKey = SnapshotInfo.getTableKey(volumeName, bucketName,
+          snapshotName);
+      snapshotInfo =
+          omMetadataManager.getSnapshotInfoTable().get(tableKey);
+
+      if (snapshotInfo == null) {
+        // Snapshot does not exist
+        throw new OMException("Snapshot does not exist", FILE_NOT_FOUND);
+      }
+
+      if (!snapshotInfo.getSnapshotStatus().equals(
+          SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE)) {
+        // If the snapshot is not in active state, throw exception as well
+        throw new OMException("Snapshot exists but no longer active",

Review Comment:
   Perhaps "Snapshot pending deletion".



-- 
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 diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1092366077


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the
+    // exact same timestamp.
+    OMRequest.Builder omRequestBuilder = omRequest.toBuilder()
+        .setDeleteSnapshotRequest(
+            DeleteSnapshotRequest.newBuilder()
+                .setVolumeName(volumeName)
+                .setBucketName(bucketName)
+                .setSnapshotName(snapshotName)
+                .setDeletionTime(Time.now()));
+
+    return omRequestBuilder.build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumSnapshotDeletes();
+
+//    boolean acquiredBucketLock = false;
+    boolean acquiredSnapshotLock = false;
+    IOException exception = null;
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+        getOmRequest());
+    OMClientResponse omClientResponse = null;
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    UserInfo userInfo = getOmRequest().getUserInfo();
+
+    final DeleteSnapshotRequest request =
+        getOmRequest().getDeleteSnapshotRequest();
+
+    final String volumeName = request.getVolumeName();
+    final String bucketName = request.getBucketName();
+    final String snapshotName = request.getSnapshotName();
+    final long deletionTime = request.getDeletionTime();
+
+    SnapshotInfo snapshotInfo = null;
+
+    try {
+      // TODO: Do we need a bucket lock here? Probably not?

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] hemantk-12 commented on a diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1092489550


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the

Review Comment:
   I meant we are setting deletion time at line 100. According to me, this timestamp might not be the same for leader and follower if it matters.
   
   Sorry, I was not clear in the previous comment.



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

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

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


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


[GitHub] [ozone] smengcl commented on a diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1092577393


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the

Review Comment:
   As discussed offline, the approach of setting the value in omRequest would work, similar to what we have done in `S3GetSecretRequest` [here](https://github.com/apache/ozone/blob/dcd17390bef1b050e0d79aee3d70411aa69238ef/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/s3/security/S3GetSecretRequest.java#L111-L119).



-- 
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 #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl merged PR #4175:
URL: https://github.com/apache/ozone/pull/4175


-- 
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 #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

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

   @GeorgeJahad @hemanthboyina @aswinshakil Please also take a look.
   
   Some of the dead code commented out is intentional in this draft to facilitate discussion. e.g. whether to keep the `BUCKET_LOCK` or not in `OMSnapshotDeleteRequest`. They will be cleaned up later.


-- 
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 diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1092374753


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the

Review Comment:
   Hi Hemant, not sure if I understand your concern here, but `DeleteSnapshotRequest` does not depend on snapshotId with the current approach.
   
   Regarding `omRequest`, we are building on top of the incoming request with `omRequest.toBuilder()` and rewriting `DeleteSnapshotRequest` params only. Other request params should have been retained as-is (e.g. `UserInfo`).



-- 
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 diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "smengcl (via GitHub)" <gi...@apache.org>.
smengcl commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1088259915


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the
+    // exact same timestamp.
+    OMRequest.Builder omRequestBuilder = omRequest.toBuilder()
+        .setDeleteSnapshotRequest(
+            DeleteSnapshotRequest.newBuilder()
+                .setVolumeName(volumeName)
+                .setBucketName(bucketName)
+                .setSnapshotName(snapshotName)
+                .setDeletionTime(Time.now()));
+
+    return omRequestBuilder.build();
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long transactionLogIndex,
+      OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
+
+    OMMetrics omMetrics = ozoneManager.getMetrics();
+    omMetrics.incNumSnapshotDeletes();
+
+//    boolean acquiredBucketLock = false;
+    boolean acquiredSnapshotLock = false;
+    IOException exception = null;
+    OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
+
+    OMResponse.Builder omResponse = OmResponseUtil.getOMResponseBuilder(
+        getOmRequest());
+    OMClientResponse omClientResponse = null;
+    AuditLogger auditLogger = ozoneManager.getAuditLogger();
+
+    UserInfo userInfo = getOmRequest().getUserInfo();
+
+    final DeleteSnapshotRequest request =
+        getOmRequest().getDeleteSnapshotRequest();
+
+    final String volumeName = request.getVolumeName();
+    final String bucketName = request.getBucketName();
+    final String snapshotName = request.getSnapshotName();
+    final long deletionTime = request.getDeletionTime();
+
+    SnapshotInfo snapshotInfo = null;
+
+    try {
+      // TODO: Do we need a bucket lock here? Probably not?
+//      acquiredBucketLock =
+//          omMetadataManager.getLock().acquireReadLock(BUCKET_LOCK,
+//              volumeName, bucketName);
+
+      acquiredSnapshotLock =
+          omMetadataManager.getLock().acquireWriteLock(SNAPSHOT_LOCK,
+              volumeName, bucketName, snapshotName);
+
+      // Retrieve SnapshotInfo from the table
+      String tableKey = SnapshotInfo.getTableKey(volumeName, bucketName,
+          snapshotName);
+      snapshotInfo =
+          omMetadataManager.getSnapshotInfoTable().get(tableKey);
+
+      if (snapshotInfo == null) {

Review Comment:
   The previous check is done in `preExecute`. Thus we have to check it again here in `validateAndUpdateCache` because stages of different requests can interleave.



-- 
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] hemantk-12 commented on a diff in pull request #4175: HDDS-6857. [Snapshot] Implement Snapshot Delete CLI and API

Posted by "hemantk-12 (via GitHub)" <gi...@apache.org>.
hemantk-12 commented on code in PR #4175:
URL: https://github.com/apache/ozone/pull/4175#discussion_r1084679662


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1678,6 +1681,13 @@ message SnapshotDiffRequest {
   required string toSnapshot = 4;
 }
 
+message DeleteSnapshotRequest {
+  optional string volumeName = 1;
+  optional string bucketName = 2;
+  optional string snapshotName = 3;
+  optional uint64 deletionTime = 4;

Review Comment:
   What's the usage of this? Is it for scheduled delete? Or just for audit purpose? 



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotDeleteRequest.java:
##########
@@ -0,0 +1,217 @@
+/**
+ * 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.request.snapshot;
+
+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.OmUtils;
+import org.apache.hadoop.ozone.audit.AuditLogger;
+import org.apache.hadoop.ozone.audit.OMAction;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OMMetrics;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotDeleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteSnapshotResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.UserInfo;
+import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FILE_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.SNAPSHOT_LOCK;
+
+/**
+ * Handles DeleteSnapshot Request.
+ */
+public class OMSnapshotDeleteRequest extends OMClientRequest {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotDeleteRequest.class);
+
+  public OMSnapshotDeleteRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+//  @DisallowedUntilLayoutVersion(OZONE_SNAPSHOT_SCHEMA)
+  public OMRequest preExecute(OzoneManager ozoneManager) throws IOException {
+
+    final OMRequest omRequest = super.preExecute(ozoneManager);
+
+    final DeleteSnapshotRequest deleteSnapshotRequest =
+        omRequest.getDeleteSnapshotRequest();
+
+    final String snapshotName = deleteSnapshotRequest.getSnapshotName();
+    // Verify snapshot name. TODO: Can remove
+    OmUtils.validateSnapshotName(snapshotName);
+
+    String volumeName = deleteSnapshotRequest.getVolumeName();
+    String bucketName = deleteSnapshotRequest.getBucketName();
+
+    // Permission check
+    UserGroupInformation ugi = createUGI();
+    String bucketOwner = ozoneManager.getBucketOwner(volumeName, bucketName,
+        IAccessAuthorizer.ACLType.READ, OzoneObj.ResourceType.BUCKET);
+    if (!ozoneManager.isAdmin(ugi) &&
+        !ozoneManager.isOwner(ugi, bucketOwner)) {
+      throw new OMException(
+          "Only bucket owners and Ozone admins can delete snapshots",
+          OMException.ResultCodes.PERMISSION_DENIED);
+    }
+
+    // Set deletion time here so OM leader and follower would have the

Review Comment:
   Not sure if this would work. I had to add snapshotId before submitting request to Raits.
   
   https://github.com/apache/ozone/blob/1ee083178df192d48b16c04b663384f7c336c374/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerProtocolServerSideTranslatorPB.java#L139 and https://github.com/apache/ozone/blob/1ee083178df192d48b16c04b663384f7c336c374/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/validation/RequestValidations.java#L124
   
   
   
   Please double check 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