You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "xBis7 (via GitHub)" <gi...@apache.org> on 2023/06/02 10:33:34 UTC

[GitHub] [ozone] xBis7 opened a new pull request, #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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

   ## What changes were proposed in this pull request?
   
   This PR adds an option to the `ozone sh snapshot snapshotDiff` command, to cancel an IN_PROGRESS snapshotDiff job. If the option is used and the job is IN_PROGRESS, then the status is updated to CANCELED. 
   
   The part of the code that might take up the most resources and cause the delay, has been refactored so that we can keep checking if the JobStatus is CANCELED before every method call. 
   
   If the job is canceled, then the method doing the calculations returns and the job remains CANCELED until the `SnapshotDiffCleanupService` deletes it from the snapDiffJobTable and the user can resubmit it. 
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-8490
   
   ## How was this patch tested?
   
   This patch was tested with new integration tests with and without using the MiniOzoneCluster. It was also tested manually using the docker dev environment.
   


-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -389,9 +399,19 @@ public SnapshotDiffResponse getSnapshotDiffReport(
     String snapDiffJobKey = fsInfo.getSnapshotID() + DELIMITER +
         tsInfo.getSnapshotID();
 
+    // Check if the job already exists in the table
+    // before getting report status.
+    if (cancel && Objects.nonNull(snapDiffJobTable.get(snapDiffJobKey)) &&

Review Comment:
   > Also snapDiff job submission and cancellation should be consider two separate options in OmSnapshotManager and SnapshotDiffManager.
   > 
   > In [OzoneManager](https://github.com/apache/ozone/pull/4819/files#diff-54bbfc5b5dd89a4d035bee206c53d3d11bbb579a2957abca589e313283d98af9R4523), you can use cancel to decide which flow needs to be called.
   
   The OzoneManager changes will certainly make everything look cleaner but won't change anything.
   
   > 1. If job doesn't exist, we would create a new job. Which is not needed. Correct behavior should be return empty response with a message.
   
   Are you saying that if the job doesn't exist then cancel should just return and not proceed with submitting the job? Maybe return CANCELED response? Why submit a new job with a cancel parameter? Or should we check for the case that the job is stuck as QUEUED?
   
   > 2. If job is not IN_PROGRESS status and it is DONE, it will not mark the job cancel and will return first page response. I am not sure if that is correct response to client.
   
   I thought that's the intention here. The user wants to cancel a job only if it's in progress. If the job is DONE, then why cancel it? The system has already gone through the whole process and finished with it. 
   
   > 3. It will fail at line 409, if two or more commands are issued at the same time for the same snapDiff job because one of them will mark it cancel successfully and other/s will fail due to [check](https://github.com/apache/ozone/blob/9034434285f3877246b3ebc6a7513c082931b279/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java#L1164).
   
   If the job is not IN_PROGRESS then it shouldn't be marked as canceled. I don't think the exception affects us in any case. 
   
   We try to update the job status and
   * <b>cancel succeeds</b>, job status is updated to CANCELED. 
   Cancel check should pick it up but in any case where somehow we end up updating status to DONE or FAILED or REJECTED, we get an exception, status is not updated and the job is left in the table as CANCELED. In that case The cleanup service picks it up and deletes everything.
   * <b>cancel fails</b>, job status is not updated to CANCELED. 
   Job status is either FAILED, REJECTED or DONE. If it's DONE then great it will return the diff report to the user. If it's one of the other options, the job will be cleaned up by the cleanup service.
   
   The way I see the flow when we run `getSnapshotDiffReport()` with cancel=true, 
   If the job is
   * NEW, cancel is ignored, job is registered as QUEUED and then submitted
   * QUEUED, cancel is ignored, gets submitted again
   * IN_PROGRESS, cancel takes effect, job is marked as canceled
   * DONE, cancel is ignored, get the diff report 
   * FAILED - REJECTED - CANCELED, cancel is ignored, get the response until the cleanup service deletes the job.
   
   This all seems intended behavior to me.  



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1799,10 +1800,22 @@ message SnapshotDiffResponse {
     DONE = 3;
     REJECTED = 4;
     FAILED = 5;
+    CANCELED = 6;
   }
+
+  enum CancelStatusProto {

Review Comment:
   Yes, you are right. I just couldn't think of a better name. I'll make the changes.



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1799,10 +1800,22 @@ message SnapshotDiffResponse {
     DONE = 3;
     REJECTED = 4;
     FAILED = 5;
+    CANCELED = 6;
   }
+
+  enum CancelStatusProto {

Review Comment:
   IMO this is more of just a `Result` to the one-time client request rather than a `Status` for the diff job.
   
   ```suggestion
     enum JobCancelResultProto {
   ```
   
   Another example:
   
   https://github.com/apache/ozone/blob/bdd26eeca9fe7adfd87a4ee1e0acfd84d6148c57/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto#L182-L199



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java:
##########
@@ -0,0 +1,302 @@
+/**
+ * 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.snapshot;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+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.OzoneConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmTestManagers;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse.JobStatus;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.rocksdb.RocksDBException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.DELIMITER;
+
+/**
+ * Tests for {@link SnapshotDiffManager}.
+ */
+public class TestSnapshotDiffManager {
+
+  private static final String VOLUME = "vol";
+  private static final String BUCKET = "bucket";
+
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+  private static OMMetadataManager omMetadataManager;
+  private static SnapshotDiffManager snapshotDiffManager;
+  private static PersistentMap<String, SnapshotDiffJob> snapDiffJobTable;
+
+  @BeforeAll
+  public static void init() throws AuthenticationException,
+      IOException, RocksDBException {
+    metaDir = GenericTestUtils.getRandomizedTestDir();

Review Comment:
   I'm surprised and curious what is causing this failure. `@TempDir` is used at some many other place. Were you trying to cleanup or delete it? 



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

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

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


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -627,6 +628,47 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    // Cancel works only if the job is IN_PROGRESS, and
+    // it's ignored if the job has any other status.
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Cancel here is ignored, job gets saved in the snapDiffJobTable
+    // as QUEUED and then transitions IN_PROGRESS.
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());
+
+    // Executing the command again should return CANCELED,
+    // until the job is picked up by the SnapshotDiffCleanupService
+    // and removed from the snapDiffJobTable.
+    response = store.snapshotDiff(volumeName,

Review Comment:
   I can add it here as well. IMO, they are both integration tests since in `TestSnapshotDiffManager` we are not mocking anything, we are using `TestOmManagers` which creates an actual OzoneManager. The difference is that this test spins up a whole cluster to run the test on.
   
   We had a previous discussion in a sync call about this. Most of Ozone tests are integration tests, but we keep calling them unit tests because they don't use MiniOzoneCluster. 



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java:
##########
@@ -0,0 +1,302 @@
+/**
+ * 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.snapshot;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+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.OzoneConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmTestManagers;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse.JobStatus;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.rocksdb.RocksDBException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.DELIMITER;
+
+/**
+ * Tests for {@link SnapshotDiffManager}.
+ */
+public class TestSnapshotDiffManager {
+
+  private static final String VOLUME = "vol";
+  private static final String BUCKET = "bucket";
+
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+  private static OMMetadataManager omMetadataManager;
+  private static SnapshotDiffManager snapshotDiffManager;
+  private static PersistentMap<String, SnapshotDiffJob> snapDiffJobTable;
+
+  @BeforeAll
+  public static void init() throws AuthenticationException,
+      IOException, RocksDBException {
+    metaDir = GenericTestUtils.getRandomizedTestDir();

Review Comment:
   > Were you trying to cleanup or delete it?
   
   No, the annotation is supposed to take care of creation and cleanup. Check below,
   
   https://github.com/xBis7/ozone/blob/a575a094515f75efe95b9dfda8c9a8bf1de0eb3a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java#L69-L70
   
   https://github.com/xBis7/ozone/blob/0694343dae2a3f0237eae4e30b0e5359dde3da75/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java#L75-L76



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/snapshot/SnapshotDiffResponse.java:
##########
@@ -43,16 +45,59 @@ public static JobStatus fromProtobuf(JobStatusProto jobStatusProto) {
     }
   }
 
+  /**
+   * Snapshot diff cancel status enum.
+   */
+  public enum CancelStatus {
+    JOB_NOT_CANCELED("Job is not canceled"),

Review Comment:
   Yes, I'll change the description to `Job hasn't been cancelled` to make it more explanatory. 



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java:
##########
@@ -0,0 +1,302 @@
+/**
+ * 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.snapshot;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+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.OzoneConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmTestManagers;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse.JobStatus;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.rocksdb.RocksDBException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.DELIMITER;
+
+/**
+ * Tests for {@link SnapshotDiffManager}.
+ */
+public class TestSnapshotDiffManager {
+
+  private static final String VOLUME = "vol";
+  private static final String BUCKET = "bucket";
+
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+  private static OMMetadataManager omMetadataManager;
+  private static SnapshotDiffManager snapshotDiffManager;
+  private static PersistentMap<String, SnapshotDiffJob> snapDiffJobTable;
+
+  @BeforeAll
+  public static void init() throws AuthenticationException,
+      IOException, RocksDBException {
+    metaDir = GenericTestUtils.getRandomizedTestDir();

Review Comment:
   @hemantk-12 I had multiple workflows failing due to `@tempdir` not being able to cleanup properly and I found online that other people had the same experience. 
   
   These are some of the failing workflows. They all belong to a dummy branch I had for cancelling a snapshot diff job.
   
   https://github.com/xBis7/ozone/actions/runs/5147594598/jobs/9268500272#step:6:2565
   
   https://github.com/xBis7/ozone/actions/runs/5147452987/jobs/9267857940#step:6:2675
   
   https://github.com/xBis7/ozone/actions/runs/5143362416/jobs/9258280943#step:6:2562



-- 
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] xBis7 commented on pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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

   @smengcl @hemantk-12 Thanks 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] smengcl commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -631,6 +639,127 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, false);
+
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());

Review Comment:
   Hmm. I'm think if we could intercept the `snapDiffExecutor`:
   
   https://github.com/apache/ozone/blob/9c6cd4b93e0427fc12bbedae8156c746cbe6cea1/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java#L216-L224
   
   Or just add a spin lock just for the tests inside `generateSnapshotDiffReport()`.



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -631,6 +639,127 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, false);
+
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());

Review Comment:
   Could this be a source of flakiness? There is no guarantee that the diff job isn't DONE when calling the second `snapshotDiff` with `cancel = true` right?
   
   Can fix this if there is a way to suspend the SnapshotDiff worker thread before the first call.



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -631,6 +639,127 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, false);
+
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());

Review Comment:
   Could this be a source of flakiness? There is no guarantee that the diff job is still running when calling the second `snapshotDiff` with `cancel = true` right?
   
   Can fix this if there is a way to suspend the SnapshotDiff worker thread before the first call.



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1799,10 +1800,22 @@ message SnapshotDiffResponse {
     DONE = 3;
     REJECTED = 4;
     FAILED = 5;
+    CANCELED = 6;
   }
+
+  enum CancelStatusProto {

Review Comment:
   IMO this is more of just a `Result` to the one-time client request rather than a `Status` for the diff job like `JobStatusProto`.
   
   ```suggestion
     enum JobCancelResultProto {
   ```
   
   Another example:
   
   https://github.com/apache/ozone/blob/bdd26eeca9fe7adfd87a4ee1e0acfd84d6148c57/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto#L182-L199



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -627,6 +628,47 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    // Cancel works only if the job is IN_PROGRESS, and
+    // it's ignored if the job has any other status.

Review Comment:
   This comment is not needed.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -627,6 +628,47 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    // Cancel works only if the job is IN_PROGRESS, and
+    // it's ignored if the job has any other status.
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Cancel here is ignored, job gets saved in the snapDiffJobTable
+    // as QUEUED and then transitions IN_PROGRESS.
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());
+
+    // Executing the command again should return CANCELED,
+    // until the job is picked up by the SnapshotDiffCleanupService
+    // and removed from the snapDiffJobTable.
+    response = store.snapshotDiff(volumeName,

Review Comment:
   As a completeness for this test, can we call [SnapshotDiffCleanupService.run()](https://github.com/apache/ozone/blob/master/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDiffCleanupService.java#L112), and assert that it is moved to cleanup table?



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -1010,6 +1066,11 @@ private long generateDiffReport(
       try (ClosableIterator<byte[]>
                objectIdsIterator = objectIDsToCheck.iterator()) {
         while (objectIdsIterator.hasNext()) {
+          if (snapDiffJobTable.get(jobKey).getStatus()

Review Comment:
   1. To avoid NPE.
   ```suggestion
             if (snapDiffJobTable.get(jobKey) != null && 
                 CANCELED.equals(snapDiffJobTable.get(jobKey).getStatus())) {
               return -1L;
             }
   ```
   
   2. I feel it would be too expensive to check job status for every entry. Should we check after like every 100 entries? 



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java:
##########
@@ -0,0 +1,302 @@
+/**
+ * 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.snapshot;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+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.OzoneConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmTestManagers;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse.JobStatus;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.rocksdb.RocksDBException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.DELIMITER;
+
+/**
+ * Tests for {@link SnapshotDiffManager}.
+ */
+public class TestSnapshotDiffManager {
+
+  private static final String VOLUME = "vol";
+  private static final String BUCKET = "bucket";
+
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+  private static OMMetadataManager omMetadataManager;
+  private static SnapshotDiffManager snapshotDiffManager;
+  private static PersistentMap<String, SnapshotDiffJob> snapDiffJobTable;
+
+  @BeforeAll
+  public static void init() throws AuthenticationException,
+      IOException, RocksDBException {
+    metaDir = GenericTestUtils.getRandomizedTestDir();

Review Comment:
   You can use @TempDir annotation to create temp dir for the test.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -718,36 +746,63 @@ private void generateSnapshotDiffReport(final String jobKey,
       Table<String, OmKeyInfo> tsKeyTable = toSnapshot.getMetadataManager()
           .getKeyTable(bucketLayout);
 
-      getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsKeyTable, tsKeyTable,
-          fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
-          tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
-          objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
-          path.toString());
-
-      if (bucketLayout.isFileSystemOptimized()) {
-        validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-            toSnapshotName);
-
-        Table<String, OmDirectoryInfo> fsDirTable =
-            fromSnapshot.getMetadataManager().getDirectoryTable();
-        Table<String, OmDirectoryInfo> tsDirTable =
-            toSnapshot.getMetadataManager().getDirectoryTable();
-
-        getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsDirTable, tsDirTable,
-            fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
-            tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
-            objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
-            path.toString());
-      }
-
-      validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-          toSnapshotName);
-      long totalDiffEntries = generateDiffReport(jobId,
-          objectIDsToCheckMap,
-          objectIdToKeyNameMapForFromSnapshot,
-          objectIdToKeyNameMapForToSnapshot);
+      // These are the most time and resource consuming method calls.
+      // Split the calls into steps and store them in an array, to avoid
+      // repetition while constantly checking if the job is cancelled.
+      Callable<Void>[] methodCalls = new Callable[]{
+          () -> {
+            getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsKeyTable, tsKeyTable,
+                fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
+                tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
+                objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
+                path.toString());
+            return null;
+          },
+          () -> {
+            if (bucketLayout.isFileSystemOptimized()) {
+              validateSnapshotsAreActive(volumeName, bucketName,

Review Comment:
   You can move `validateSnapshotsAreActive` check from here and line 780 to callable check in for loop 799-805. 



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -389,9 +399,19 @@ public SnapshotDiffResponse getSnapshotDiffReport(
     String snapDiffJobKey = fsInfo.getSnapshotID() + DELIMITER +
         tsInfo.getSnapshotID();
 
+    // Check if the job already exists in the table
+    // before getting report status.
+    if (cancel && Objects.nonNull(snapDiffJobTable.get(snapDiffJobKey)) &&

Review Comment:
   @hemantk-12 We can have a `CancelStatus` like [this](https://github.com/xBis7/ozone/blob/HDDS-8490-exp/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/snapshot/SnapshotDiffResponse.java#L52-L78) and based on cancel's success or failure, print the corresponding description on the client by modifying the `toString()` method like [this](https://github.com/xBis7/ozone/blob/HDDS-8490-exp/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/snapshot/SnapshotDiffResponse.java#L121-L137).
   
   If this approach isn't acceptable, we can be more invasive and create a new `SnapshotDiffResponse` class just for cancel.
   
   Let me know what you think.



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffJob.java:
##########
@@ -61,6 +62,7 @@ public SnapshotDiffJob(long creationTime,
                          String fromSnapshot,
                          String toSnapshot,
                          boolean forceFullDiff,
+                         boolean cancel,

Review Comment:
   Isn't `jobStatus` enough to tell if job is cancelled or not?



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -389,9 +399,19 @@ public SnapshotDiffResponse getSnapshotDiffReport(
     String snapDiffJobKey = fsInfo.getSnapshotID() + DELIMITER +
         tsInfo.getSnapshotID();
 
+    // Check if the job already exists in the table
+    // before getting report status.
+    if (cancel && Objects.nonNull(snapDiffJobTable.get(snapDiffJobKey)) &&

Review Comment:
   There are few problems by adding this to current creation flow.
   1. If job doesn't exist, we would create a new job. Which is not needed. Correct behavior should be return empty response with a message.
   1. If job is not `IN_PROGRESS` status and it is `DONE`, it will not mark the job cancel and will return first page response. I am not sure if that is correct response to client.
   1. It will fail at line 409, if two or more commands are issued at the same time for the same snapDiff job because one of them will mark it cancel successfully and other/s will fail due to [check](https://github.com/apache/ozone/blob/9034434285f3877246b3ebc6a7513c082931b279/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java#L1164).
   
   I think it would be better to create a separate flow for cancel and handle these cases appropriately. Also snapDiff job submission and cancellation should be consider two separate options in `OmSnapshotManager` and `SnapshotDiffManager`.
   
   In [OzoneManager](https://github.com/apache/ozone/pull/4819/files#diff-54bbfc5b5dd89a4d035bee206c53d3d11bbb579a2957abca589e313283d98af9R4523), you can use cancel to decide which flow needs to be called.
   
   ```
     public SnapshotDiffResponse snapshotDiff(String volume,
                                              String bucket,
                                              String fromSnapshot,
                                              String toSnapshot,
                                              String token,
                                              int pageSize,
                                              boolean forceFullDiff,
                                              boolean cancel)
         throws IOException {
       if (cancel) {
         return omSnapshotManager.cancelSnapshotDiff(volume, bucket,
             fromSnapshot, toSnapshot);
       } else {
         return omSnapshotManager.getSnapshotDiffReport(volume, bucket,
             fromSnapshot, toSnapshot, token, pageSize, forceFullDiff);
       }
     }
   ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -422,6 +442,12 @@ public SnapshotDiffResponse getSnapshotDiffReport(
               bucketName, fromSnapshotName, toSnapshotName, new ArrayList<>(),
               null),
           REJECTED, defaultWaitTime);
+    case CANCELED:
+      return new SnapshotDiffResponse(
+          new SnapshotDiffReportOzone(snapshotRoot.toString(), volumeName,
+              bucketName, fromSnapshotName, toSnapshotName, new ArrayList<>(),
+              null),
+          CANCELED, defaultWaitTime);

Review Comment:
   Return zero instead of `defaultWaitTime` here.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -718,36 +746,60 @@ private void generateSnapshotDiffReport(final String jobKey,
       Table<String, OmKeyInfo> tsKeyTable = toSnapshot.getMetadataManager()
           .getKeyTable(bucketLayout);
 
-      getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsKeyTable, tsKeyTable,
-          fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
-          tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
-          objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
-          path.toString());
-
-      if (bucketLayout.isFileSystemOptimized()) {
-        validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-            toSnapshotName);
-
-        Table<String, OmDirectoryInfo> fsDirTable =
-            fromSnapshot.getMetadataManager().getDirectoryTable();
-        Table<String, OmDirectoryInfo> tsDirTable =
-            toSnapshot.getMetadataManager().getDirectoryTable();
-
-        getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsDirTable, tsDirTable,
-            fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
-            tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
-            objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
-            path.toString());
-      }
+      // These are the most time and resource consuming method calls.
+      // Split the calls into steps and store them in an array, to avoid
+      // repetition while constantly checking if the job is cancelled.
+      Callable<Void>[] methodCalls = new Callable[]{
+          () -> {
+            getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsKeyTable, tsKeyTable,
+                fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
+                tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
+                objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
+                path.toString());
+            return null;
+          },
+          () -> {
+            if (bucketLayout.isFileSystemOptimized()) {
+              Table<String, OmDirectoryInfo> fsDirTable =
+                  fromSnapshot.getMetadataManager().getDirectoryTable();
+              Table<String, OmDirectoryInfo> tsDirTable =
+                  toSnapshot.getMetadataManager().getDirectoryTable();
+
+              getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsDirTable, tsDirTable,
+                  fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
+                  tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
+                  objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
+                  path.toString());
+            }
+            return null;
+          },
+          () -> {
+            long totalDiffEntries = generateDiffReport(jobKey,
+                jobId,
+                objectIDsToCheckMap,
+                objectIdToKeyNameMapForFromSnapshot,
+                objectIdToKeyNameMapForToSnapshot);
+            // If job is canceled, totalDiffEntries will be equal to -1.
+            if (totalDiffEntries >= 0 &&
+                !snapDiffJobTable.get(jobKey)
+                    .getStatus().equals(CANCELED)) {
+              updateJobStatusToDone(jobKey, totalDiffEntries);
+            }
+            return null;
+          }
+      };
 
-      validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-          toSnapshotName);
-      long totalDiffEntries = generateDiffReport(jobId,
-          objectIDsToCheckMap,
-          objectIdToKeyNameMapForFromSnapshot,
-          objectIdToKeyNameMapForToSnapshot);
+      // Check if the job is cancelled, before every method call.
+      for (Callable<Void> methodCall : methodCalls) {
+        if (snapDiffJobTable.get(jobKey).getStatus()
+            .equals(CANCELED)) {
+          return;
+        }

Review Comment:
   Can you please move this to another function `validateSnapshotDiffJobIsActive`? An reuse it at other place like lines 1071-1073 and lines 784-786.



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -627,6 +628,47 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    // Cancel works only if the job is IN_PROGRESS, and
+    // it's ignored if the job has any other status.
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Cancel here is ignored, job gets saved in the snapDiffJobTable
+    // as QUEUED and then transitions IN_PROGRESS.
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());
+
+    // Executing the command again should return CANCELED,
+    // until the job is picked up by the SnapshotDiffCleanupService
+    // and removed from the snapDiffJobTable.
+    response = store.snapshotDiff(volumeName,

Review Comment:
   I will add it here as well. IMO, they are both integration tests since in `TestSnapshotDiffManager` we are not mocking anything, we are using `TestOmManagers` which creates an actual OzoneManager. The difference is that this test spins up a whole cluster to run the test on.
   
   We had a previous discussion in a sync call about this. Most of Ozone tests are integration tests, they create entire objects instead of mocking them but we keep calling them unit tests because they don't use MiniOzoneCluster. 



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -389,9 +399,19 @@ public SnapshotDiffResponse getSnapshotDiffReport(
     String snapDiffJobKey = fsInfo.getSnapshotID() + DELIMITER +
         tsInfo.getSnapshotID();
 
+    // Check if the job already exists in the table
+    // before getting report status.
+    if (cancel && Objects.nonNull(snapDiffJobTable.get(snapDiffJobKey)) &&

Review Comment:
   @hemantk-12 We can have a `CancelStatus` like [this](https://github.com/apache/ozone/blob/838a4ea8ce447a37b38c603cfa78b4993498d7a1/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/snapshot/SnapshotDiffResponse.java#L52-L78) and based on cancel's success or failure, print the corresponding description on the client by modifying the `toString()` method like [this](https://github.com/apache/ozone/blob/838a4ea8ce447a37b38c603cfa78b4993498d7a1/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/snapshot/SnapshotDiffResponse.java#L121-L137).
   
   If this approach isn't acceptable, we can be more invasive and create a new `SnapshotDiffResponse` class just for cancel.
   
   Let me know what you think.



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -389,9 +399,19 @@ public SnapshotDiffResponse getSnapshotDiffReport(
     String snapDiffJobKey = fsInfo.getSnapshotID() + DELIMITER +
         tsInfo.getSnapshotID();
 
+    // Check if the job already exists in the table
+    // before getting report status.
+    if (cancel && Objects.nonNull(snapDiffJobTable.get(snapDiffJobKey)) &&

Review Comment:
   > Also snapDiff job submission and cancellation should be consider two separate options in OmSnapshotManager and SnapshotDiffManager.
   > 
   > In [OzoneManager](https://github.com/apache/ozone/pull/4819/files#diff-54bbfc5b5dd89a4d035bee206c53d3d11bbb579a2957abca589e313283d98af9R4523), you can use cancel to decide which flow needs to be called.
   
   The OzoneManager changes will certainly make everything look cleaner but won't change anything.
   
   > 1. If job doesn't exist, we would create a new job. Which is not needed. Correct behavior should be return empty response with a message.
   
   Are you saying that if the job doesn't exist then cancel should just return and not proceed with submitting the job? Maybe return CANCELED response? Why submit a new job with a cancel parameter? Or should we check for the case that the job is stuck as QUEUED?
   
   > 2. If job is not IN_PROGRESS status and it is DONE, it will not mark the job cancel and will return first page response. I am not sure if that is correct response to client.
   
   I thought that's the intention here. The user wants to cancel a job only if it's in progress. If the job is DONE, then why cancel it? The system has already gone through the whole process and finished with it. 
   
   > 3. It will fail at line 409, if two or more commands are issued at the same time for the same snapDiff job because one of them will mark it cancel successfully and other/s will fail due to [check](https://github.com/apache/ozone/blob/9034434285f3877246b3ebc6a7513c082931b279/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java#L1164).
   
   If the job is not IN_PROGRESS then it shouldn't be marked as canceled. I don't think the exception affects us in any case. 
   
   We try to update the job status and
   * <b>cancel succeeds</b>, job status is updated to CANCELED. 
   Cancel check should pick it up but in any case where somehow we end up updating status to DONE or FAILED or REJECTED, we get an exception, status is not updated and the job is left in the table as CANCELED. In that case The cleanup service picks it up and deletes everything.
   * <b>cancel fails</b>, job status is not updated to CANCELED. 
   Job status is either FAILED, REJECTED or DONE. If it's DONE then great it will return the diff report to the user. If it's one of the other options, the job will be cleaned up by the cleanup service.
   
   The way I see the flow, we run `getSnapshotDiffReport()` with cancel=true. 
   If the job is
   * NEW, cancel is ignored, job is registered as QUEUED and then submitted
   * QUEUED, cancel is ignored, gets submitted again
   * IN_PROGRESS, cancel takes effect, job is marked as canceled
   * DONE, cancel is ignored, get the diff report 
   * FAILED - REJECTED - CANCELED, cancel is ignored, get the response until the cleanup service deletes the job.
   
   This all seems intended behavior to me.  



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/snapshot/SnapshotDiffHandler.java:
##########
@@ -60,6 +60,13 @@ public class SnapshotDiffHandler extends Handler {
       hidden = true)
   private boolean forceFullDiff;
 
+
+  @CommandLine.Option(names = {"-c", "--cancel"},
+      description = "Cancel a running snapshotDiff job. " +
+          "If the job is not IN_PROGRESS then the option will fail.",

Review Comment:
   ```suggestion
         description = "Request to cancel a running SnapshotDiff job. " +
             "If the job is not IN_PROGRESS, the request will fail.",
   ```



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

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

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


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffJob.java:
##########
@@ -61,6 +62,7 @@ public SnapshotDiffJob(long creationTime,
                          String fromSnapshot,
                          String toSnapshot,
                          boolean forceFullDiff,
+                         boolean cancel,

Review Comment:
   There was an idea about using the information when loading jobs on startup. But now this information seems redundant. Even if the job is cancelled, when loaded in startup it will be queued and cancel will not be enforced. I'll clean this up.



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

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

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


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -627,6 +628,47 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    // Cancel works only if the job is IN_PROGRESS, and
+    // it's ignored if the job has any other status.
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Cancel here is ignored, job gets saved in the snapDiffJobTable
+    // as QUEUED and then transitions IN_PROGRESS.
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());
+
+    // Executing the command again should return CANCELED,
+    // until the job is picked up by the SnapshotDiffCleanupService
+    // and removed from the snapDiffJobTable.
+    response = store.snapshotDiff(volumeName,

Review Comment:
   I will add it here as well. IMO, they are both integration tests since in `TestSnapshotDiffManager` we are not mocking anything, we are using `TestOmManagers` which creates an actual OzoneManager. The difference is that this test spins up a whole cluster to run the test on.
   
   We had a previous discussion in a sync call about this. Most of Ozone tests are integration tests, but we keep calling them unit tests because they don't use MiniOzoneCluster. 



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffJob.java:
##########
@@ -61,6 +62,7 @@ public SnapshotDiffJob(long creationTime,
                          String fromSnapshot,
                          String toSnapshot,
                          boolean forceFullDiff,
+                         boolean cancel,

Review Comment:
   There was an idea about using the information when loading jobs on startup. But now this information seems redundant. I'll clean this up.



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

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

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


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


[GitHub] [ozone] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -389,9 +399,19 @@ public SnapshotDiffResponse getSnapshotDiffReport(
     String snapDiffJobKey = fsInfo.getSnapshotID() + DELIMITER +
         tsInfo.getSnapshotID();
 
+    // Check if the job already exists in the table
+    // before getting report status.
+    if (cancel && Objects.nonNull(snapDiffJobTable.get(snapDiffJobKey)) &&

Review Comment:
   > Also snapDiff job submission and cancellation should be consider two separate options in OmSnapshotManager and SnapshotDiffManager.
   > 
   > In [OzoneManager](https://github.com/apache/ozone/pull/4819/files#diff-54bbfc5b5dd89a4d035bee206c53d3d11bbb579a2957abca589e313283d98af9R4523), you can use cancel to decide which flow needs to be called.
   
   The OzoneManager changes will certainly make everything look cleaner but won't change anything.
   
   > 1. If job doesn't exist, we would create a new job. Which is not needed. Correct behavior should be return empty response with a message.
   
   Are you saying that if the job doesn't exist then cancel should just return and not proceed with submitting the job? Maybe return `CANCELED` response? Why submit a new job with a cancel parameter? Or should we check for the case that the job is stuck as `QUEUED`?
   
   > 2. If job is not IN_PROGRESS status and it is DONE, it will not mark the job cancel and will return first page response. I am not sure if that is correct response to client.
   
   I thought that's the intention here. The user wants to cancel a job only if it's in progress. If the job is `DONE`, then why cancel it? The system has already gone through the whole process and finished with it. 
   
   > 3. It will fail at line 409, if two or more commands are issued at the same time for the same snapDiff job because one of them will mark it cancel successfully and other/s will fail due to [check](https://github.com/apache/ozone/blob/9034434285f3877246b3ebc6a7513c082931b279/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java#L1164).
   
   If the job is not `IN_PROGRESS` then it shouldn't be marked as canceled. I don't think the exception affects us in any case. 
   
   We try to update the job status and
   * <b>cancel succeeds</b>, job status is updated to `CANCELED`. 
   Cancel check should pick it up but in any case where somehow we end up updating status to `DONE` or `FAILED` or `REJECTED`, we get an exception, status is not updated and the job is left in the table as `CANCELED`. In that case The cleanup service picks it up and deletes everything.
   * <b>cancel fails</b>, job status is not updated to `CANCELED`. 
   Job status is either `FAILED`, `REJECTED` or `DONE`. If it's `DONE` then great it will return the diff report to the user. If it's one of the other options, the job will be cleaned up by the cleanup service.
   
   The way I see the flow when we run `getSnapshotDiffReport()` with cancel=true, 
   If the job is
   * `NEW`, cancel is ignored, job is registered as `QUEUED` and then submitted
   * `QUEUED`, cancel is ignored, job gets submitted again
   * `IN_PROGRESS`, cancel takes effect, job is marked as canceled
   * `DONE`, cancel is ignored, get the diff report 
   * `FAILED` - `REJECTED` - `CANCELED`, cancel is ignored, get the response until the cleanup service deletes the job.
   
   This all seems intended behavior to me.  



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java:
##########
@@ -670,15 +670,17 @@ private List<OzoneSnapshot> getNextListOfSnapshots(String prevSnapshot)
    * @return the difference report between two snapshots

Review Comment:
   new param `cancel` should be added to the javadoc as well.



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

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

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


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


[GitHub] [ozone] smengcl commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1799,10 +1800,22 @@ message SnapshotDiffResponse {
     DONE = 3;
     REJECTED = 4;
     FAILED = 5;
+    CANCELED = 6;
   }
+
+  enum CancelStatusProto {
+    JOB_NOT_CANCELED = 1;

Review Comment:
   Let's use double-l
   ```suggestion
       JOB_NOT_CANCELLED = 1;
   ```



##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1799,10 +1800,22 @@ message SnapshotDiffResponse {
     DONE = 3;
     REJECTED = 4;
     FAILED = 5;
+    CANCELED = 6;
   }
+
+  enum CancelStatusProto {
+    JOB_NOT_CANCELED = 1;
+    NEW_JOB = 2;
+    JOB_DONE = 3;
+    INVALID_STATUS_TRANSITION = 4;
+    JOB_ALREADY_CANCELED = 5;

Review Comment:
   nit
   ```suggestion
       JOB_ALREADY_CANCELLED = 5;
   ```



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDiffCleanupService.java:
##########
@@ -124,6 +125,16 @@ public void run() {
     moveOldSnapDiffJobsToPurgeTable();
   }
 
+  @VisibleForTesting
+  public byte[] getEntryFromPurgedJobTable(String jobId) {
+    try {
+      return db.get().get(snapDiffPurgedJobCfh,
+          codecRegistry.asRawData(jobId));
+    } catch (IOException | RocksDBException e) {
+      throw new RuntimeException(e);

Review Comment:
   ```suggestion
         // TODO: [SNAPSHOT] Fail gracefully.
         throw new RuntimeException(e);
   ```



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -389,9 +399,19 @@ public SnapshotDiffResponse getSnapshotDiffReport(
     String snapDiffJobKey = fsInfo.getSnapshotID() + DELIMITER +
         tsInfo.getSnapshotID();
 
+    // Check if the job already exists in the table
+    // before getting report status.
+    if (cancel && Objects.nonNull(snapDiffJobTable.get(snapDiffJobKey)) &&

Review Comment:
   @hemantk-12 We can have a `CancelStatus` like [this](https://github.com/xBis7/ozone/blob/d2dd7b94e36260c1d6a418911c0feea6ee94109f/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/snapshot/SnapshotDiffResponse.java#L51-L77) and based on cancel's success or failure, print the corresponding description on the client by modifying the `toString()` method like [this](https://github.com/xBis7/ozone/blob/d2dd7b94e36260c1d6a418911c0feea6ee94109f/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/snapshot/SnapshotDiffResponse.java#L120-L138).
   
   If this approach isn't acceptable, we can be more invasive and create a new `SnapshotDiffResponse` class just for cancel.
   
   Let me know what you think.



-- 
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] xBis7 commented on pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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

   @hemantk-12 @smengcl Can you take a look at this PR?


-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -1010,6 +1066,11 @@ private long generateDiffReport(
       try (ClosableIterator<byte[]>
                objectIdsIterator = objectIDsToCheck.iterator()) {
         while (objectIdsIterator.hasNext()) {
+          if (snapDiffJobTable.get(jobKey).getStatus()

Review Comment:
   > To avoid NPE.
   
   Yes, the job might have failed or being rejected and got removed from the table.
   
   > Should we check after like every 100 entries?
   
   Sure, I'll fix that.



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -718,36 +746,63 @@ private void generateSnapshotDiffReport(final String jobKey,
       Table<String, OmKeyInfo> tsKeyTable = toSnapshot.getMetadataManager()
           .getKeyTable(bucketLayout);
 
-      getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsKeyTable, tsKeyTable,
-          fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
-          tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
-          objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
-          path.toString());
-
-      if (bucketLayout.isFileSystemOptimized()) {
-        validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-            toSnapshotName);
-
-        Table<String, OmDirectoryInfo> fsDirTable =
-            fromSnapshot.getMetadataManager().getDirectoryTable();
-        Table<String, OmDirectoryInfo> tsDirTable =
-            toSnapshot.getMetadataManager().getDirectoryTable();
-
-        getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsDirTable, tsDirTable,
-            fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
-            tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
-            objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
-            path.toString());
-      }
-
-      validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-          toSnapshotName);
-      long totalDiffEntries = generateDiffReport(jobId,
-          objectIDsToCheckMap,
-          objectIdToKeyNameMapForFromSnapshot,
-          objectIdToKeyNameMapForToSnapshot);
+      // These are the most time and resource consuming method calls.
+      // Split the calls into steps and store them in an array, to avoid
+      // repetition while constantly checking if the job is cancelled.
+      Callable<Void>[] methodCalls = new Callable[]{
+          () -> {
+            getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsKeyTable, tsKeyTable,
+                fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
+                tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
+                objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
+                path.toString());
+            return null;
+          },
+          () -> {
+            if (bucketLayout.isFileSystemOptimized()) {
+              validateSnapshotsAreActive(volumeName, bucketName,

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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffJob.java:
##########
@@ -61,6 +62,7 @@ public SnapshotDiffJob(long creationTime,
                          String fromSnapshot,
                          String toSnapshot,
                          boolean forceFullDiff,
+                         boolean cancel,

Review Comment:
   There was an idea about using the information when loading jobs on startup. But now this information seems redundant. Even if the job is cancelled and we were to take some action, when loaded in startup it will be queued and cancel will not be enforced. I'll clean this up.



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

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

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


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


[GitHub] [ozone] xBis7 commented on pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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

   @hemantk-12 I've addressed all your comments and updated the patch to send different responses back to the client in case job cancelling fails. I've also updated the existing tests and added new ones as well. Let me know how it looks.


-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -631,6 +639,127 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, false);
+
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());

Review Comment:
   Could this be a source of flakiness? There is no guarantee that the diff job is still running when calling the second `snapshotDiff` with `cancel = true` right?



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1799,10 +1800,22 @@ message SnapshotDiffResponse {
     DONE = 3;
     REJECTED = 4;
     FAILED = 5;
+    CANCELED = 6;
   }
+
+  enum CancelStatusProto {

Review Comment:
   IMO this is more of a `Result` to the one-time client request rather than a `Status` for the diff job.
   
   ```suggestion
     enum JobCancelResultProto {
   ```
   
   e.g. https://github.com/apache/ozone/blob/bdd26eeca9fe7adfd87a4ee1e0acfd84d6148c57/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto#L182-L199



##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1799,10 +1800,22 @@ message SnapshotDiffResponse {
     DONE = 3;
     REJECTED = 4;
     FAILED = 5;
+    CANCELED = 6;
   }
+
+  enum CancelStatusProto {

Review Comment:
   IMO this is more of a `Result` to the one-time client request rather than a `Status` for the diff job.
   
   ```suggestion
     enum JobCancelResultProto {
   ```
   
   Another example:
   
   https://github.com/apache/ozone/blob/bdd26eeca9fe7adfd87a4ee1e0acfd84d6148c57/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto#L182-L199



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1799,10 +1800,22 @@ message SnapshotDiffResponse {
     DONE = 3;
     REJECTED = 4;
     FAILED = 5;
+    CANCELED = 6;
   }
+
+  enum CancelStatusProto {

Review Comment:
   IMO this is more of just a `Result`/`Response` to the one-time client request rather than a `Status` for the diff job like `JobStatusProto`.
   
   ```suggestion
     enum JobCancelResultProto {
   ```
   
   Another example:
   
   https://github.com/apache/ozone/blob/bdd26eeca9fe7adfd87a4ee1e0acfd84d6148c57/hadoop-hdds/interface-server/src/main/proto/ScmServerProtocol.proto#L182-L199



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/snapshot/SnapshotDiffResponse.java:
##########
@@ -67,18 +112,28 @@ public long getWaitTimeInMs() {
     return waitTimeInMs;
   }
 
+  public CancelStatus getCancelStatus() {
+    return cancelStatus;
+  }
+
   @Override
   public String toString() {
     StringBuilder str = new StringBuilder();
-    if (jobStatus == JobStatus.DONE) {
-      str.append(snapshotDiffReport.toString());
+    if (cancelStatus == CancelStatus.JOB_NOT_CANCELED ||
+        cancelStatus == CancelStatus.CANCEL_SUCCESS) {
+      if (jobStatus == JobStatus.DONE) {
+        str.append(snapshotDiffReport.toString());
+      } else {
+        str.append("Snapshot diff job is ");
+        str.append(jobStatus);
+        str.append("\n");

Review Comment:
   nit
   ```suggestion
           str.append(". ");
   ```



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -631,6 +639,127 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, false);
+
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());

Review Comment:
   > Can fix this if there is a way to suspend the SnapshotDiff worker thread before the first call.
   
   That's a good idea, but the first call is submitting the job and we would be suspending the thread for a job that doesn't even exist in the snap diff table. If we do it after the submitting then there is no difference with cancelling the job. 
   
   The only way I can think of is calling the methods that perform all these operations and do the suspension between the calls instead of using the ObjectStore api. But that is done in `TestSnapshotDiffManager`. 
   
   So far this test hasn't proved to be flaky. Usually, the workflow takes too long to run. 
   
   
   
   



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -627,6 +628,47 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    // Cancel works only if the job is IN_PROGRESS, and
+    // it's ignored if the job has any other status.
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Cancel here is ignored, job gets saved in the snapDiffJobTable
+    // as QUEUED and then transitions IN_PROGRESS.
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());
+
+    // Executing the command again should return CANCELED,
+    // until the job is picked up by the SnapshotDiffCleanupService
+    // and removed from the snapDiffJobTable.
+    response = store.snapshotDiff(volumeName,

Review Comment:
   @hemantk-12 This is testing snapDiff cancel from a client perspective using the ObjectStore. IMO, this check is more suitable for `TestSnapshotDiffManager` which is checking tables and such. I placed it there. Let me know what you think. 



-- 
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] xBis7 commented on pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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

   @hemantk-12 Thanks for reviewing this, I've addressed all your comments.


-- 
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] xBis7 commented on pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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

   @smengcl Thanks for review, I've addressed all your comments and made the changes. Pending the comment about the test and its flakiness. 


-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -627,6 +628,47 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    // Cancel works only if the job is IN_PROGRESS, and
+    // it's ignored if the job has any other status.

Review Comment:
   I added it to explain it doesn't make any difference to remove cancel a new job. I removed it.



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

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

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


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


[GitHub] [ozone] smengcl commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java:
##########
@@ -717,13 +717,15 @@ default List<SnapshotInfo> listSnapshot(
    * @return the difference report between two snapshots

Review Comment:
   Same here



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

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

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


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


[GitHub] [ozone] smengcl commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -710,43 +787,67 @@ private void generateSnapshotDiffReport(final String jobKey,
 
       boolean useFullDiff = snapshotForceFullDiff || forceFullDiff;
 
-      validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-          toSnapshotName);
+      if (!validateDiffJobAndSnapshotsAreActive(volumeName,
+          bucketName, fromSnapshotName, toSnapshotName)) {
+        return;
+      }
       Table<String, OmKeyInfo> fsKeyTable = fromSnapshot.getMetadataManager()
           .getKeyTable(bucketLayout);
       Table<String, OmKeyInfo> tsKeyTable = toSnapshot.getMetadataManager()
           .getKeyTable(bucketLayout);
 
-      getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsKeyTable, tsKeyTable,
-          fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
-          tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
-          objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
-          path.toString());
-
-      if (bucketLayout.isFileSystemOptimized()) {
-        validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-            toSnapshotName);
-
-        Table<String, OmDirectoryInfo> fsDirTable =
-            fromSnapshot.getMetadataManager().getDirectoryTable();
-        Table<String, OmDirectoryInfo> tsDirTable =
-            toSnapshot.getMetadataManager().getDirectoryTable();
-
-        getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsDirTable, tsDirTable,
-            fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
-            tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
-            objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
-            path.toString());
-      }
-
-      validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-          toSnapshotName);
-      long totalDiffEntries = generateDiffReport(jobId,
-          objectIDsToCheckMap,
-          objectIdToKeyNameMapForFromSnapshot,
-          objectIdToKeyNameMapForToSnapshot);
+      // These are the most time and resource consuming method calls.
+      // Split the calls into steps and store them in an array, to avoid
+      // repetition while constantly checking if the job is cancelled.
+      Callable<Void>[] methodCalls = new Callable[]{
+          () -> {

Review Comment:
   nice



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -418,6 +478,12 @@ public SnapshotDiffResponse getSnapshotDiffReport(
               bucketName, fromSnapshotName, toSnapshotName, new ArrayList<>(),
               null),
           REJECTED, defaultWaitTime);
+    case CANCELED:
+      return new SnapshotDiffResponse(
+          new SnapshotDiffReportOzone(snapshotRoot.toString(), volumeName,
+              bucketName, fromSnapshotName, toSnapshotName, new ArrayList<>(),
+              null),
+          CANCELED, 0L, CancelStatus.CANCEL_SUCCESS);

Review Comment:
   Unrelated to this PR but I think `SnapshotDiffReportOzone` could use a `Builder` subclass. cc @hemantk-12 



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -623,20 +689,29 @@ private synchronized SnapshotDiffJob getSnapDiffReportStatus(
     return snapDiffJob;
   }
 
-  private void validateSnapshotsAreActive(final String volumeName,
-                                          final String bucketName,
-                                          final String fromSnapshotName,
-                                          final String toSnapshotName)
+  private boolean validateDiffJobAndSnapshotsAreActive(

Review Comment:
   ```suggestion
     private boolean areDiffJobAndSnapshotsActive(
   ```
   
   since it returns `boolean` now



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/snapshot/SnapshotDiffResponse.java:
##########
@@ -43,16 +45,59 @@ public static JobStatus fromProtobuf(JobStatusProto jobStatusProto) {
     }
   }
 
+  /**
+   * Snapshot diff cancel status enum.
+   */
+  public enum CancelStatus {
+    JOB_NOT_CANCELED("Job is not canceled"),

Review Comment:
   IIUC this indicates "Job has never been cancelled"?



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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

   Thanks @xBis7 for the PR. Thanks @hemantk-12 for reviewing this.
   
   We can file a follow-up jira to deal with the potential flakiness if needed.


-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -627,6 +628,47 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    // Cancel works only if the job is IN_PROGRESS, and
+    // it's ignored if the job has any other status.
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Cancel here is ignored, job gets saved in the snapDiffJobTable
+    // as QUEUED and then transitions IN_PROGRESS.
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());
+
+    // Executing the command again should return CANCELED,
+    // until the job is picked up by the SnapshotDiffCleanupService
+    // and removed from the snapDiffJobTable.
+    response = store.snapshotDiff(volumeName,

Review Comment:
   I agree and disagree at the same time :P. The reason I wanted it here because this is integration test and should test end-end flow. While `TestSnapshotDiffManager` is unit test and should just test `SnapshotDiffManager` as unit. 



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -627,6 +628,47 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    // Cancel works only if the job is IN_PROGRESS, and
+    // it's ignored if the job has any other status.
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Cancel here is ignored, job gets saved in the snapDiffJobTable
+    // as QUEUED and then transitions IN_PROGRESS.
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());
+
+    // Executing the command again should return CANCELED,
+    // until the job is picked up by the SnapshotDiffCleanupService
+    // and removed from the snapDiffJobTable.
+    response = store.snapshotDiff(volumeName,

Review Comment:
   @hemantk-12 In this class `preFinalizationChecks();` always fails locally but passes remotely, should we do something about that? I'm not saying in this PR.



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestSnapshotDiffManager.java:
##########
@@ -0,0 +1,322 @@
+/**
+ * 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.snapshot;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+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.OzoneConfigKeys;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmTestManagers;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.service.SnapshotDiffCleanupService;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse.JobStatus;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
+import org.apache.ozone.test.GenericTestUtils;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.rocksdb.RocksDBException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.DELIMITER;
+
+/**
+ * Tests for {@link SnapshotDiffManager}.
+ */
+public class TestSnapshotDiffManager {
+
+  private static final String VOLUME = "vol";
+  private static final String BUCKET = "bucket";
+
+  private static File metaDir;
+  private static OzoneManager ozoneManager;
+  private static OMMetadataManager omMetadataManager;
+  private static SnapshotDiffManager snapshotDiffManager;
+  private static PersistentMap<String, SnapshotDiffJob> snapDiffJobTable;
+  private static SnapshotDiffCleanupService snapshotDiffCleanupService;
+
+  @BeforeAll
+  public static void init() throws AuthenticationException,
+      IOException, RocksDBException {
+    metaDir = GenericTestUtils.getRandomizedTestDir();
+    if (!metaDir.exists()) {
+      Assertions.assertTrue(metaDir.mkdirs());
+    }
+    OzoneConfiguration conf = new OzoneConfiguration();
+    conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS,
+        metaDir.getAbsolutePath());
+
+    OmTestManagers omTestManagers = new OmTestManagers(conf);
+    ozoneManager = omTestManagers.getOzoneManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+
+    snapshotDiffManager = ozoneManager
+        .getOmSnapshotManager().getSnapshotDiffManager();
+    snapDiffJobTable = snapshotDiffManager.getSnapDiffJobTable();
+    snapshotDiffCleanupService = ozoneManager
+        .getOmSnapshotManager().getSnapshotDiffCleanupService();
+
+    createVolumeAndBucket();
+  }
+
+  @AfterAll
+  public static void cleanUp() {
+    FileUtil.fullyDelete(metaDir);
+  }
+
+  /**
+   * Test Snapshot Diff job cancellation.
+   * Cancel is ignored unless the job is IN_PROGRESS.
+   *
+   * Once a job is canceled, it stays in the table until
+   * SnapshotDiffCleanupService removes it.
+   *
+   * Job response until that happens, is CANCELED.
+   */
+  @Test
+  public void testCanceledSnapshotDiffReport()
+      throws IOException {
+    String fromSnapshotName = "snap-" + RandomStringUtils.randomNumeric(5);
+    String toSnapshotName = "snap-" + RandomStringUtils.randomNumeric(5);
+    String fromSnapshotId = UUID.randomUUID().toString();
+    String toSnapshotId = UUID.randomUUID().toString();
+    String diffJobKey = fromSnapshotId + DELIMITER + toSnapshotId;
+
+    setUpKeysAndSnapshots(fromSnapshotName, toSnapshotName,
+        fromSnapshotId, toSnapshotId);
+
+    SnapshotDiffJob diffJob = snapDiffJobTable.get(diffJobKey);
+    Assertions.assertNull(diffJob);
+
+    // This is a new job, cancel should be ignored.
+    SnapshotDiffResponse snapshotDiffResponse = snapshotDiffManager
+        .getSnapshotDiffReport(VOLUME, BUCKET,
+            fromSnapshotName, toSnapshotName,
+            0, 0, false, true);
+
+    // Response should be IN_PROGRESS
+    Assertions.assertEquals(JobStatus.IN_PROGRESS,
+        snapshotDiffResponse.getJobStatus());
+
+    // Check snapDiffJobTable.
+    diffJob = snapDiffJobTable.get(diffJobKey);
+    Assertions.assertNotNull(diffJob);
+    // Status stored in the table should be IN_PROGRESS.
+    Assertions.assertEquals(JobStatus.IN_PROGRESS,
+        diffJob.getStatus());
+
+    // Job should be canceled.
+    snapshotDiffResponse = snapshotDiffManager
+        .getSnapshotDiffReport(VOLUME, BUCKET,
+            fromSnapshotName, toSnapshotName,
+            0, 0, false, true);
+
+    // Response should be CANCELED.
+    Assertions.assertEquals(JobStatus.CANCELED,
+        snapshotDiffResponse.getJobStatus());
+
+    // Check snapDiffJobTable.
+    diffJob = snapDiffJobTable.get(diffJobKey);
+    Assertions.assertNotNull(diffJob);
+    // Status stored in the table should be CANCELED.
+    Assertions.assertEquals(JobStatus.CANCELED,
+        diffJob.getStatus());
+
+    // Job hasn't been removed from the
+    // table yet and response should still be canceled.
+    snapshotDiffResponse = snapshotDiffManager
+        .getSnapshotDiffReport(VOLUME, BUCKET,
+            fromSnapshotName, toSnapshotName,
+            0, 0, false, true);
+
+    // Response should be CANCELED.
+    Assertions.assertEquals(JobStatus.CANCELED,
+        snapshotDiffResponse.getJobStatus());
+
+    // Check snapDiffJobTable.
+    diffJob = snapDiffJobTable.get(diffJobKey);
+    Assertions.assertNotNull(diffJob);
+    // Status stored in the table should be CANCELED.
+    Assertions.assertEquals(JobStatus.CANCELED,
+        diffJob.getStatus());
+
+    String jobId = diffJob.getJobId();
+
+    // Job is still in the snapDiffJobTable,
+    // so it shouldn't exist in the purged job table.
+    Assertions.assertNull(snapshotDiffCleanupService
+        .getEntryFromPurgedJobTable(jobId));

Review Comment:
   @hemantk-12 I added the purged job table check here.



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

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

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


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


[GitHub] [ozone] smengcl commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java:
##########
@@ -1070,10 +1070,11 @@ List<OzoneSnapshot> listSnapshot(
    * @return the difference report between two snapshots

Review Comment:
   and here



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

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

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


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


[GitHub] [ozone] smengcl commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1799,10 +1800,22 @@ message SnapshotDiffResponse {
     DONE = 3;
     REJECTED = 4;
     FAILED = 5;
+    CANCELED = 6;
   }
+
+  enum CancelStatusProto {
+    JOB_NOT_CANCELED = 1;
+    NEW_JOB = 2;
+    JOB_DONE = 3;
+    INVALID_STATUS_TRANSITION = 4;
+    JOB_ALREADY_CANCELED = 5;
+    CANCEL_SUCCESS = 6;

Review Comment:
   nit
   ```suggestion
       CANCELLATION_SUCCESS = 6;
   ```



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/snapshot/SnapshotDiffResponse.java:
##########
@@ -43,16 +45,59 @@ public static JobStatus fromProtobuf(JobStatusProto jobStatusProto) {
     }
   }
 
+  /**
+   * Snapshot diff cancel status enum.
+   */
+  public enum CancelStatus {
+    JOB_NOT_CANCELED("Job is not canceled"),
+    NEW_JOB("Cannot cancel a newly submitted job"),
+    JOB_DONE("Job is DONE, cancel failed"),

Review Comment:
   ```suggestion
       JOB_DONE("Job is already DONE"),
   ```



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

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

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


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


[GitHub] [ozone] smengcl commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmSnapshotManager.java:
##########
@@ -489,6 +489,15 @@ public static String getOzonePathKeyWithVolumeBucketNames(
     return OM_KEY_PREFIX + volumeId + OM_KEY_PREFIX + bucketId + OM_KEY_PREFIX;
   }
 
+  @VisibleForTesting
+  public SnapshotDiffManager getSnapshotDiffManager() {
+    return snapshotDiffManager;
+  }
+
+  @VisibleForTesting
+  public SnapshotDiffCleanupService getSnapshotDiffCleanupService() {
+    return snapshotDiffCleanupService;
+  }

Review Comment:
   ```suggestion
     }
   
   ```



-- 
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 #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/snapshot/SnapshotDiffResponse.java:
##########
@@ -43,16 +45,59 @@ public static JobStatus fromProtobuf(JobStatusProto jobStatusProto) {
     }
   }
 
+  /**
+   * Snapshot diff cancel status enum.
+   */
+  public enum CancelStatus {

Review Comment:
   Similarly
   ```suggestion
     public enum JobCancelResult {
   ```



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/snapshot/SnapshotDiffManager.java:
##########
@@ -710,43 +787,67 @@ private void generateSnapshotDiffReport(final String jobKey,
 
       boolean useFullDiff = snapshotForceFullDiff || forceFullDiff;
 
-      validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-          toSnapshotName);
+      if (!validateDiffJobAndSnapshotsAreActive(volumeName,
+          bucketName, fromSnapshotName, toSnapshotName)) {
+        return;
+      }
       Table<String, OmKeyInfo> fsKeyTable = fromSnapshot.getMetadataManager()
           .getKeyTable(bucketLayout);
       Table<String, OmKeyInfo> tsKeyTable = toSnapshot.getMetadataManager()
           .getKeyTable(bucketLayout);
 
-      getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsKeyTable, tsKeyTable,
-          fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
-          tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
-          objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
-          path.toString());
-
-      if (bucketLayout.isFileSystemOptimized()) {
-        validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-            toSnapshotName);
-
-        Table<String, OmDirectoryInfo> fsDirTable =
-            fromSnapshot.getMetadataManager().getDirectoryTable();
-        Table<String, OmDirectoryInfo> tsDirTable =
-            toSnapshot.getMetadataManager().getDirectoryTable();
-
-        getDeltaFilesAndDiffKeysToObjectIdToKeyMap(fsDirTable, tsDirTable,
-            fromSnapshot, toSnapshot, fsInfo, tsInfo, useFullDiff,
-            tablePrefixes, objectIdToKeyNameMapForFromSnapshot,
-            objectIdToKeyNameMapForToSnapshot, objectIDsToCheckMap,
-            path.toString());
-      }
-
-      validateSnapshotsAreActive(volumeName, bucketName, fromSnapshotName,
-          toSnapshotName);
-      long totalDiffEntries = generateDiffReport(jobId,
-          objectIDsToCheckMap,
-          objectIdToKeyNameMapForFromSnapshot,
-          objectIdToKeyNameMapForToSnapshot);
+      // These are the most time and resource consuming method calls.
+      // Split the calls into steps and store them in an array, to avoid
+      // repetition while constantly checking if the job is cancelled.
+      Callable<Void>[] methodCalls = new Callable[]{
+          () -> {

Review Comment:
   thanks



-- 
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] xBis7 commented on a diff in pull request #4819: HDDS-8490. [Snapshot] Ability to cancel an in-progress snapdiff job

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


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmSnapshot.java:
##########
@@ -631,6 +639,127 @@ public void testSnapDiff() throws Exception {
 
   }
 
+  @Test
+  public void testSnapDiffCancel() throws Exception {
+    // Create key1 and take snapshot.
+    String key1 = "key-1-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key1);
+    String fromSnapName = "snap-1-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, fromSnapName);
+
+    // Create key2 and take snapshot.
+    String key2 = "key-2-" + RandomStringUtils.randomNumeric(5);
+    createFileKey(ozoneBucket, key2);
+    String toSnapName = "snap-2-" + RandomStringUtils.randomNumeric(5);
+    createSnapshot(volumeName, bucketName, toSnapName);
+
+    SnapshotDiffResponse response = store.snapshotDiff(
+        volumeName, bucketName, fromSnapName, toSnapName,
+        null, 0, false, false);
+
+    assertEquals(IN_PROGRESS, response.getJobStatus());
+
+    response = store.snapshotDiff(volumeName,
+        bucketName, fromSnapName, toSnapName,
+        null, 0, false, true);
+
+    // Job status should be updated to CANCELED.
+    assertEquals(CANCELED, response.getJobStatus());

Review Comment:
   > Can fix this if there is a way to suspend the SnapshotDiff worker thread before the first call.
   
   That's a good idea, but the first call is submitting the job and we would be suspending the thread for a job that doesn't even exist in the snap diff table. If we do it after the submitting then there is no difference with cancelling the job. 
   
   The only way I can think of is calling the methods that perform all these operations and do the suspension between the calls instead of using the ObjectStore api.
   
   
   
   



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