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

[GitHub] [ozone] aswinshakil opened a new pull request, #4244: HDDS-7740. Implement SnapshotDeletingService

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

   ## What changes were proposed in this pull request?
   
   The patch implements `SnapshotDeletingService`, it goes through the deleted snapshot's `deletedTable` and does either of the following.
   
   1. Move it to the next non-deleted snapshot, if there is none move it to active object store DB.
   2. Or Move it to the active object store DB.
   
   ## Follow-up TODO
   Right now the `SnapshotDeletingService` doesn't handle the following and will be done in the next patch. 
   Tracked here: [HDDS-7883](https://issues.apache.org/jira/browse/HDDS-7883)
   
   1. Handle the cleanup of renamed keys between snapshots.
   2. To accommodate keys from FSO buckets. 
   3. Cleaning up SnapshotChain, Checkpoint directory. `OMSnapshotPurgeRequest` will do these cleanups. 
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-7740
   
   ## How was this patch tested?
   
   The patch was tested with UTs and manual testing. 
   


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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              KeyValuePair.Builder toActiveDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+              KeyValuePair.Builder toNextDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, nextSnapshot,
+                toActiveDBList, toNextDBList);
+            snapshotLimit--;
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+
+        successRunCount.incrementAndGet();
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(KeyValuePair.Builder toActiveDb,
+        KeyValuePair.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        SnapshotInfo nextSnapshot, List<KeyValuePair> toActiveDBList,
+        List<KeyValuePair> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      if (nextSnapshot != null) {
+        moveDeletedKeysBuilder.setNextSnapshot(nextSnapshot.getProtobuf());
+      }
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)

Review Comment:
   It iterates on path based chain which is same as bucket specific. 



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();

Review Comment:
   Should seek to the first key in that bucket. Updating the PR to do so. 



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java:
##########
@@ -757,6 +758,7 @@ private void instantiateServices(boolean withNewSnapshot) throws IOException {
     omMetadataReader = new OmMetadataReader(keyManager, prefixManager,
         this, LOG, AUDIT, metrics);
     omSnapshotManager = new OmSnapshotManager(this);
+    snapshotChainManager = new SnapshotChainManager(metadataManager);

Review Comment:
   @neils-dev mentioned this should be placed in `OmMetadataManagerImpl`



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java:
##########
@@ -262,6 +268,24 @@ public void start(OzoneConfiguration configuration) {
         snapshotSstFilteringService.start();
       }
     }
+
+    if (snapshotDeletingService == null) {
+      long snapshotServiceInterval = configuration.getTimeDuration(
+          OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+          OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL_DEFAULT,
+          TimeUnit.MILLISECONDS);
+      long snapshotServiceTimeout = configuration.getTimeDuration(
+          OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+          OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT_DEFAULT,
+          TimeUnit.MILLISECONDS);
+      try {
+        snapshotDeletingService = new SnapshotDeletingService(
+            snapshotServiceInterval, snapshotServiceTimeout, ozoneManager);
+        snapshotDeletingService.start();

Review Comment:
   So this should only run on the leader OM 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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1691,6 +1694,18 @@ message DeleteSnapshotRequest {
   optional uint64 deletionTime = 4;
 }
 
+message SnapshotMoveDeletedKeysRequest {
+  required SnapshotInfo fromSnapshot = 1;

Review Comment:
   Context: https://github.com/apache/ozone/pull/4120#issuecomment-1397672766



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3180,6 +3188,25 @@
     </description>
   </property>
 
+  <property>
+    <name>ozone.snapshot.deletion.service.timeout</name>
+    <value>300s</value>
+    <tag>OZONE, PERFORMANCE, OM</tag>
+    <description>
+      Timeout value for SnapshotDeletingService.
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.snapshot.deletion.service.interval</name>
+    <value>30s</value>

Review Comment:
   QQ: Does `SnapshotDeletingService` task run at fixed rate or fixed delay?



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Handles OMSnapshotMoveDeletedKeys Request.
+ */
+public class OMSnapshotMoveDeletedKeysRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotMoveDeletedKeysRequest.class);
+
+  public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+    OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager();
+
+    SnapshotMoveDeletedKeysRequest moveDeletedKeysRequest =
+        getOmRequest().getSnapshotMoveDeletedKeysRequest();
+
+    // If there is no Non-Deleted Snapshot move the
+    // keys to Active Object Store.
+    SnapshotInfo nextSnapshot = null;
+    if (moveDeletedKeysRequest.hasNextSnapshot()) {
+      nextSnapshot = SnapshotInfo
+          .getFromProtobuf(moveDeletedKeysRequest.getNextSnapshot());
+    }
+
+    List<KeyValuePair> activeDBKeysList =
+        moveDeletedKeysRequest.getActiveDBKeysList();
+    List<KeyValuePair> nextDBKeysList =
+        moveDeletedKeysRequest.getNextDBKeysList();
+
+    OmSnapshot omFromSnapshot = null;
+    OmSnapshot omNextSnapshot = null;
+
+    try {

Review Comment:
   nit:
   ```suggestion
       if (nextSnapshot != null) {
         try {
           omNextSnapshot = (OmSnapshot) omSnapshotManager
               .checkForSnapshot(nextSnapshot.getVolumeName(),
                   nextSnapshot.getBucketName(),
                   getSnapshotPrefix(nextSnapshot.getName()));
         } catch (IOException ex) {
           LOG.error("Error occurred when moving keys between snapshots", ex);
         }
       }
   ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -49,6 +49,7 @@ public class SnapshotChainManager {
       snapshotChainPath;
   private Map<String, String> latestPathSnapshotID;
   private String latestGlobalSnapshotID;
+  private Map<String, String> snapshotPathToKey;

Review Comment:
   What is key here? It would be better if you specify that in variable name.



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+

Review Comment:
   Please remove unnecessary line.



##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3165,6 +3165,14 @@
       sst filtering service per time interval.
     </description>
   </property>
+  <property>
+    <name>ozone.snapshot.deleting.limit.per.task</name>
+    <value>10</value>
+    <tag>OZONE, PERFORMANCE, OM</tag>
+    <description>The number of snapshots to be reclaimed by the

Review Comment:
   ```suggestion
       <description>The number of maximum snapshots to be reclaimed by the
         Snapshot Deleting Service per run.
    ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -91,8 +93,8 @@ private void addSnapshotGlobal(String snapshotID,
   };
 
   private void addSnapshotPath(String snapshotPath,
-                               String snapshotID,
-                               String prevPathID) throws IOException {
+      String snapshotID, String prevPathID, String snapTableKey)

Review Comment:
   Also alignment is off. Previous alignment was correct.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<KeyValuePair> activeDBKeysList;
+  private List<KeyValuePair> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<KeyValuePair> activeDBKeysList,
+       List<KeyValuePair> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (KeyValuePair activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (KeyValuePair nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,
+                nextDBKey.getKey(), nextDBOmKeyInfo);
+      } else {
+        omMetadataManager.getDeletedTable()
+            .put(nextDBKey.getKey(), nextDBOmKeyInfo);
+      }
+    }
+  }
+
+  private RepeatedOmKeyInfo createRepeatedOmKeyInfo(List<KeyInfo> keyInfosList)

Review Comment:
   I think it should be either `keyInfoList` or `keyInfos`.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {

Review Comment:
   nit:
   ```suggestion
         if (StringUtils.isEmpty(latestPathSnapshot)) {
   ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<KeyValuePair> activeDBKeysList;
+  private List<KeyValuePair> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<KeyValuePair> activeDBKeysList,
+       List<KeyValuePair> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (KeyValuePair activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (KeyValuePair nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,
+                nextDBKey.getKey(), nextDBOmKeyInfo);
+      } else {
+        omMetadataManager.getDeletedTable()
+            .put(nextDBKey.getKey(), nextDBOmKeyInfo);
+      }
+    }
+  }
+
+  private RepeatedOmKeyInfo createRepeatedOmKeyInfo(List<KeyInfo> keyInfosList)
+      throws IOException {
+    RepeatedOmKeyInfo result = null;
+
+    for (KeyInfo keyInfo: keyInfosList) {
+      if (result == null) {
+        result = new RepeatedOmKeyInfo(OmKeyInfo.getFromProtobuf(keyInfo));
+      } else {
+        result.addOmKeyInfo(OmKeyInfo.getFromProtobuf(keyInfo));
+      }
+    }
+
+    return result;
+  }
+}

Review Comment:
   Please add a new line at the end of the file.
   
   You can configure IDE to add that on saving of the file. 
   
   In IntelliJ: https://stackoverflow.com/questions/16761227/how-to-make-intellij-idea-insert-a-new-line-at-every-end-of-file



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =

Review Comment:
   Unused declaration.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Handles OMSnapshotMoveDeletedKeys Request.
+ */
+public class OMSnapshotMoveDeletedKeysRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotMoveDeletedKeysRequest.class);
+
+  public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+    OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager();
+
+    SnapshotMoveDeletedKeysRequest moveDeletedKeysRequest =
+        getOmRequest().getSnapshotMoveDeletedKeysRequest();
+
+    // If there is no Non-Deleted Snapshot move the
+    // keys to Active Object Store.
+    SnapshotInfo nextSnapshot = null;
+    if (moveDeletedKeysRequest.hasNextSnapshot()) {
+      nextSnapshot = SnapshotInfo
+          .getFromProtobuf(moveDeletedKeysRequest.getNextSnapshot());
+    }
+
+    List<KeyValuePair> activeDBKeysList =
+        moveDeletedKeysRequest.getActiveDBKeysList();
+    List<KeyValuePair> nextDBKeysList =
+        moveDeletedKeysRequest.getNextDBKeysList();
+
+    OmSnapshot omFromSnapshot = null;
+    OmSnapshot omNextSnapshot = null;
+
+    try {
+      if (nextSnapshot != null) {
+        omNextSnapshot = (OmSnapshot) omSnapshotManager
+            .checkForSnapshot(nextSnapshot.getVolumeName(),
+                nextSnapshot.getBucketName(),
+                getSnapshotPrefix(nextSnapshot.getName()));
+      }
+    } catch (IOException ex) {
+      LOG.error("Error occurred when moving keys between snapshots", ex);
+    }
+
+    OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(

Review Comment:
   Could be a one line
   ```suggestion
           OmResponseUtil.getOMResponseBuilder(getOmRequest());
    ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              KeyValuePair.Builder toActiveDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+              KeyValuePair.Builder toNextDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, nextSnapshot,
+                toActiveDBList, toNextDBList);
+            snapshotLimit--;
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+
+        successRunCount.incrementAndGet();
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(KeyValuePair.Builder toActiveDb,
+        KeyValuePair.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        SnapshotInfo nextSnapshot, List<KeyValuePair> toActiveDBList,
+        List<KeyValuePair> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      if (nextSnapshot != null) {
+        moveDeletedKeysBuilder.setNextSnapshot(nextSnapshot.getProtobuf());
+      }
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      if (chainManager.hasPreviousPathSnapshot(snapInfo.getSnapshotPath(),
+          snapInfo.getSnapshotID())) {
+        String previousPathSnapshot = chainManager.previousPathSnapshot(
+            snapInfo.getSnapshotPath(), snapInfo.getSnapshotID());
+        String tableKey = chainManager.getTableKey(previousPathSnapshot);
+        return omSnapshotManager.getSnapshotInfo(tableKey);
+      }
+      return null;
+    }
+
+    /**
+     * Get the next non deleted snapshot in the snapshot chain.
+     */
+    private SnapshotInfo getNextNonDeletedSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      while (chainManager.hasNextPathSnapshot(snapInfo.getSnapshotPath(), 
+          snapInfo.getSnapshotID())) {
+
+        String nextPathSnapshot =
+            chainManager.nextPathSnapshot(
+                snapInfo.getSnapshotPath(), snapInfo.getSnapshotID());
+
+        String tableKey = chainManager.getTableKey(nextPathSnapshot);
+        SnapshotInfo nextSnapshotInfo =
+            omSnapshotManager.getSnapshotInfo(tableKey);
+
+        if (nextSnapshotInfo.getSnapshotStatus().equals(
+            SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE)) {
+          return nextSnapshotInfo;
+        }
+      }
+      return null;
+    }
+
+    private void submitRequest(OMRequest omRequest) {
+      try {
+        if (isRatisEnabled()) {
+          OzoneManagerRatisServer server = ozoneManager.getOmRatisServer();
+
+          RaftClientRequest raftClientRequest = RaftClientRequest.newBuilder()
+              .setClientId(clientId)
+              .setServerId(server.getRaftPeerId())
+              .setGroupId(server.getRaftGroupId())
+              .setCallId(runCount.get())
+              .setMessage(Message.valueOf(
+                  OMRatisHelper.convertRequestToByteString(omRequest)))
+              .setType(RaftClientRequest.writeRequestType())
+              .build();
+
+          server.submitRequest(omRequest, raftClientRequest);
+        } else {
+          ozoneManager.getOmServerProtocol().submitRequest(null, omRequest);
+        }
+      } catch (ServiceException e) {
+        LOG.error("Snapshot Deleting request failed. " +
+            "Will retry at next run.", e);
+      }
+    }
+
+    private boolean isRatisEnabled() {
+      return ozoneManager.isRatisEnabled();
+    }
+
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+    queue.add(new SnapshotDeletingTask());
+    return queue;
+  }
+
+  private boolean shouldRun() {
+    return !suspended.get() && ozoneManager.isLeaderReady();
+  }
+
+  /**
+   * Suspend the service (for testing).
+   */
+  @VisibleForTesting

Review Comment:
   1. Is it just for testing or we will need it for `Bootstrapping slow followers/new followers.`?
   1. I don't think it is needed to specify `(for testing)` in comment since you already add `VisibleForTesting`.
   1. I think it should be package private if it is only for testing.
   



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule

Review Comment:
   nit: since this is new code, I'll suggest to use Unit-5. 



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();

Review Comment:
   ```suggestion
       if (om != null) {
         om.stop();
       }
   ```
   
   Should this be enclosed in `if`?



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+
+    //Check key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/key2");
+
+    assertNotNull(omKeyInfo1);
+
+  }
+
+  private OmKeyArgs createVolumeBucketKey(String volumeName, String bucketName,
+      BucketLayout bucketLayout, String keyName) throws IOException {
+    // cheat here, just create a volume and bucket entry so that we can
+    // create the keys, we put the same data for key and value since the
+    // system does not decode the object
+    OMRequestTestUtils.addVolumeToOM(omMetadataManager,
+        OmVolumeArgs.newBuilder()
+            .setOwnerName("owner")
+            .setAdminName("admin")
+            .setVolume(volumeName)
+            .build());
+
+    OMRequestTestUtils.addBucketToOM(omMetadataManager,
+        OmBucketInfo.newBuilder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return createKey(volumeName, bucketName, keyName);
+  }
+
+  private OmKeyArgs createKey(String volumeName, String bucketName,
+       String keyName) throws IOException {
+    OmKeyArgs keyArg =
+        new OmKeyArgs.Builder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setAcls(Collections.emptyList())
+            .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+            .setLocationInfoList(new ArrayList<>())
+            .build();
+
+    // Open and write the key without commit it.
+    OpenKeySession session = writeClient.openKey(keyArg);
+    writeClient.commitKey(keyArg, session.getId());
+
+    return keyArg;
+  }
+
+  private void createSnapshot(String volName, String bucketName,
+       String snapName, int count) throws Exception {
+    writeClient.createSnapshot(volName, bucketName, snapName);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return omMetadataManager.getSnapshotInfoTable()
+            .getEstimatedKeyCount() >= count;

Review Comment:
   Curious if `EstimatedKeyCount()` includes cache? What if cache has item but it hasn't been flush to SSTs? Would it cause uncertainty? Other places we make sure that snapshot dir exists. 



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->

Review Comment:
   nit: you can also use [Awaitility](https://github.com/awaitility/awaitility/wiki/Usage#simple).



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim

Review Comment:
   nit: Space after `//`.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setPathPreviousSnapshotID(latestPathSnapshot);
+      }
+
+      if (latestGlobalSnapshot == null || latestGlobalSnapshot.isEmpty()) {
+        snapshotInfo.setGlobalPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setGlobalPreviousSnapshotID(latestGlobalSnapshot);
+      }

Review Comment:
   +1



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");

Review Comment:
   Should we use `null` instead here? Just like in this UT:
   
   https://github.com/apache/ozone/blob/f30c0621eafc1c7f62e7df0f2da3d11ece842e33/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java#L114



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setPathPreviousSnapshotID(latestPathSnapshot);
+      }
+
+      if (latestGlobalSnapshot == null || latestGlobalSnapshot.isEmpty()) {
+        snapshotInfo.setGlobalPreviousSnapshotID("");

Review Comment:
   Same as above.



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

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

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


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


[GitHub] [ozone] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Handles OMSnapshotMoveDeletedKeys Request.
+ */
+public class OMSnapshotMoveDeletedKeysRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotMoveDeletedKeysRequest.class);
+
+  public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+    OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager();
+
+    SnapshotMoveDeletedKeysRequest moveDeletedKeysRequest =
+        getOmRequest().getSnapshotMoveDeletedKeysRequest();
+
+    // If there is no Non-Deleted Snapshot move the
+    // keys to Active Object Store.
+    SnapshotInfo nextSnapshot = null;
+    if (moveDeletedKeysRequest.hasNextSnapshot()) {
+      nextSnapshot = SnapshotInfo
+          .getFromProtobuf(moveDeletedKeysRequest.getNextSnapshot());
+    }
+

Review Comment:
   The next snapshot should be chosen from within this method, (because a next  snapshot chosen earlier may be deleted before we get here.)  See design doc for OMSnapshotMoveDeletedKeysRequest:
   
   
   https://www.notion.so/fossil-i/Design-Ozone-Snapshot-Deletion-Garbage-Collection-based-on-key-deletedTable-2a624480dc7c4bc3ad608cbf86a25541?pvs=4#58494696e28c46959d778819edd8b559
   



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1701,6 +1704,18 @@ message DeleteSnapshotRequest {
   optional uint64 deletionTime = 4;
 }
 
+message SnapshotMoveDeletedKeysRequest {
+  optional SnapshotInfo fromSnapshot = 1;
+  optional SnapshotInfo nextSnapshot = 2;

Review Comment:
   It is being used in `OMSnapshotMoveDeletedKeysResponse`



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Handles OMSnapshotMoveDeletedKeys Request.
+ */
+public class OMSnapshotMoveDeletedKeysRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotMoveDeletedKeysRequest.class);
+
+  public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+    OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    SnapshotChainManager snapshotChainManager =
+        ozoneManager.getSnapshotChainManager();
+
+    SnapshotMoveDeletedKeysRequest moveDeletedKeysRequest =
+        getOmRequest().getSnapshotMoveDeletedKeysRequest();
+    SnapshotInfo fromSnapshot = SnapshotInfo.getFromProtobuf(
+        moveDeletedKeysRequest.getFromSnapshot());
+
+    // If there is no Non-Deleted Snapshot move the
+    // keys to Active Object Store.
+    SnapshotInfo nextSnapshot = null;
+    OMClientResponse omClientResponse = null;
+    OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    try {
+      nextSnapshot = getNextActiveSnapshot(fromSnapshot,
+          snapshotChainManager, omSnapshotManager);
+
+      // Get next non-deleted snapshot.
+
+      List<SnapshotMoveKeyInfos> activeDBKeysList =
+          moveDeletedKeysRequest.getActiveDBKeysList();
+      List<SnapshotMoveKeyInfos> nextDBKeysList =
+          moveDeletedKeysRequest.getNextDBKeysList();
+
+      OmSnapshot omNextSnapshot = null;
+
+      if (nextSnapshot != null) {
+        omNextSnapshot = (OmSnapshot) omSnapshotManager
+            .checkForSnapshot(nextSnapshot.getVolumeName(),

Review Comment:
   My understanding is that validateAndUpdateCache() runs within the applyTransaction() method of the ozoneManager ratis state machine, and therefor needs to run quickly.  Is that not correct?
   



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,

Review Comment:
   Is it a problem that we are doing the put with the snapshots metadata manager, but the commit is happening to the omMetadataManager?: https://github.com/apache/ozone/blob/3de525c60c40441a938f84ffda61ebd2b8bad50b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerDoubleBuffer.java#L346-L347



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Handles OMSnapshotMoveDeletedKeys Request.
+ */
+public class OMSnapshotMoveDeletedKeysRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotMoveDeletedKeysRequest.class);
+
+  public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+    OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    SnapshotChainManager snapshotChainManager =
+        ozoneManager.getSnapshotChainManager();
+
+    SnapshotMoveDeletedKeysRequest moveDeletedKeysRequest =
+        getOmRequest().getSnapshotMoveDeletedKeysRequest();
+    SnapshotInfo fromSnapshot = SnapshotInfo.getFromProtobuf(
+        moveDeletedKeysRequest.getFromSnapshot());
+
+    // If there is no Non-Deleted Snapshot move the
+    // keys to Active Object Store.
+    SnapshotInfo nextSnapshot = null;
+    OMClientResponse omClientResponse = null;
+    OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    try {
+      OmSnapshot omFromSnapshot = (OmSnapshot) omSnapshotManager
+          .checkForSnapshot(fromSnapshot.getVolumeName(),
+              fromSnapshot.getBucketName(),
+              getSnapshotPrefix(fromSnapshot.getName()));
+
+      nextSnapshot = getNextActiveSnapshot(fromSnapshot,
+          snapshotChainManager, omSnapshotManager);
+
+      // Get next non-deleted snapshot.
+      List<SnapshotMoveKeyInfos> nextDBKeysList =
+          moveDeletedKeysRequest.getNextDBKeysList();
+      List<SnapshotMoveKeyInfos> reclaimKeysList =
+          moveDeletedKeysRequest.getReclaimKeysList();
+
+      OmSnapshot omNextSnapshot = null;
+
+      if (nextSnapshot != null) {
+        omNextSnapshot = (OmSnapshot) omSnapshotManager
+            .checkForSnapshot(nextSnapshot.getVolumeName(),
+                nextSnapshot.getBucketName(),
+                getSnapshotPrefix(nextSnapshot.getName()));
+      }
+
+      omClientResponse = new OMSnapshotMoveDeletedKeysResponse(
+          omResponse.build(), omFromSnapshot, omNextSnapshot,
+          nextDBKeysList, reclaimKeysList);
+
+    } catch (IOException ex) {
+      omClientResponse = new OMSnapshotMoveDeletedKeysResponse(
+          createErrorOMResponse(omResponse, ex));
+    } finally {
+      addResponseToDoubleBuffer(trxnLogIndex, omClientResponse,
+          omDoubleBufferHelper);
+    }
+
+    return omClientResponse;
+  }
+
+  /**
+   * Get the next non deleted snapshot in the snapshot chain.
+   */
+  private SnapshotInfo getNextActiveSnapshot(SnapshotInfo snapInfo,
+      SnapshotChainManager chainManager, OmSnapshotManager omSnapshotManager)
+      throws IOException {
+    while (chainManager.hasNextPathSnapshot(snapInfo.getSnapshotPath(),

Review Comment:
   I think it might be an infinite loop.



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1701,6 +1704,18 @@ message DeleteSnapshotRequest {
   optional uint64 deletionTime = 4;
 }
 
+message SnapshotMoveDeletedKeysRequest {
+  optional SnapshotInfo fromSnapshot = 1;
+  optional SnapshotInfo nextSnapshot = 2;

Review Comment:
   Yup it can be deleted. 



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super(SnapshotDeletingService.class.getSimpleName(), interval,
+        TimeUnit.MILLISECONDS, SNAPSHOT_DELETING_CORE_POOL_SIZE,
+        serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.isEmpty()) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+          if (bucketInfo == null) {
+            throw new OMException("Bucket " + snapInfo.getBucketName() +
+                " is not found", BUCKET_NOT_FOUND);
+          }
+
+          //TODO: [SNAPSHOT] Add lock to deletedTable and Active DB.
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<SnapshotMoveKeyInfos> toActiveDBList = new ArrayList<>();
+          List<SnapshotMoveKeyInfos> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            String snapshotBucketKey = dbBucketKey + OzoneConsts.OM_KEY_PREFIX;
+            iterator.seek(snapshotBucketKey);
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              String deletedKey = deletedKeyValue.getKey();
+
+              // Exit if it is out of the bucket scope.
+              if (!deletedKey.contains(snapshotBucketKey)) {
+                break;
+              }
+
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              SnapshotMoveKeyInfos.Builder toActiveDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+              SnapshotMoveKeyInfos.Builder toNextDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, toActiveDBList,
+                toNextDBList);
+            snapshotLimit--;
+            successRunCount.incrementAndGet();
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(SnapshotMoveKeyInfos.Builder toActiveDb,
+        SnapshotMoveKeyInfos.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        List<SnapshotMoveKeyInfos> toActiveDBList,
+        List<SnapshotMoveKeyInfos> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: [SNAPSHOT] Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)

Review Comment:
   No it doesn't. I should return the previous Snapshot regarless.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super(SnapshotDeletingService.class.getSimpleName(), interval,
+        TimeUnit.MILLISECONDS, SNAPSHOT_DELETING_CORE_POOL_SIZE,
+        serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.isEmpty()) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+          if (bucketInfo == null) {
+            throw new OMException("Bucket " + snapInfo.getBucketName() +
+                " is not found", BUCKET_NOT_FOUND);
+          }
+
+          //TODO: [SNAPSHOT] Add lock to deletedTable and Active DB.
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<SnapshotMoveKeyInfos> toActiveDBList = new ArrayList<>();
+          List<SnapshotMoveKeyInfos> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            String snapshotBucketKey = dbBucketKey + OzoneConsts.OM_KEY_PREFIX;
+            iterator.seek(snapshotBucketKey);
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              String deletedKey = deletedKeyValue.getKey();
+
+              // Exit if it is out of the bucket scope.
+              if (!deletedKey.contains(snapshotBucketKey)) {
+                break;
+              }
+
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              SnapshotMoveKeyInfos.Builder toActiveDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+              SnapshotMoveKeyInfos.Builder toNextDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, toActiveDBList,
+                toNextDBList);
+            snapshotLimit--;
+            successRunCount.incrementAndGet();
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(SnapshotMoveKeyInfos.Builder toActiveDb,
+        SnapshotMoveKeyInfos.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        List<SnapshotMoveKeyInfos> toActiveDBList,
+        List<SnapshotMoveKeyInfos> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: [SNAPSHOT] Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)

Review Comment:
   No it doesn't. I should return the previous Snapshot regardless.



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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

   Relevant UT failure:
   
   ```
   Error:  Errors: 
   Error:    TestOMSnapshotDeleteRequest.testEntryExists:272 » NullPointer
   ```


-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1691,6 +1694,18 @@ message DeleteSnapshotRequest {
   optional uint64 deletionTime = 4;
 }
 
+message SnapshotMoveDeletedKeysRequest {
+  required SnapshotInfo fromSnapshot = 1;
+  optional SnapshotInfo nextSnapshot = 2;
+  repeated KeyValuePair activeDBKeys = 3;
+  repeated KeyValuePair nextDBKeys = 4;
+}
+
+message KeyValuePair {

Review Comment:
   `KeyValuePair` sounds way too generic. Name more specifically according to its usage.
   
   Even better, put a comment around its intended usage.



##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1691,6 +1694,18 @@ message DeleteSnapshotRequest {
   optional uint64 deletionTime = 4;
 }
 
+message SnapshotMoveDeletedKeysRequest {
+  required SnapshotInfo fromSnapshot = 1;
+  optional SnapshotInfo nextSnapshot = 2;
+  repeated KeyValuePair activeDBKeys = 3;
+  repeated KeyValuePair nextDBKeys = 4;
+}
+
+message KeyValuePair {
+  required string key = 1;

Review Comment:
   ```suggestion
     optional string key = 1;
   ```



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -91,8 +93,8 @@ private void addSnapshotGlobal(String snapshotID,
   };
 
   private void addSnapshotPath(String snapshotPath,
-                               String snapshotID,
-                               String prevPathID) throws IOException {
+      String snapshotID, String prevPathID, String snapTableKey)

Review Comment:
   It'd be great if we can add javadocs to this method and the public method below.



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {

Review Comment:
   check for correct bucket.



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+

Review Comment:
   I'll add a separate integration test for this purpose. As in the UT we can't have actual data stored in DNs, only metadata changes are tested 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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java:
##########
@@ -837,6 +838,7 @@ protected OmKeyInfo wrapUncommittedBlocksAsPseudoKey(
     LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
         uncommitted, omKeyInfo.getKeyName());
     OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
+    pseudoKeyInfo.setObjectID(OBJECT_ID_DEFAULT);

Review Comment:
   Let's put a comment above this line that we are using this as a special marker to indicate that this key's blocks can be unconditionally deleted during snapshot reclamation.



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setPathPreviousSnapshotID(latestPathSnapshot);
+      }
+
+      if (latestGlobalSnapshot == null || latestGlobalSnapshot.isEmpty()) {
+        snapshotInfo.setGlobalPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setGlobalPreviousSnapshotID(latestGlobalSnapshot);
+      }

Review Comment:
   This could be encapsulated in a helper method for better code structuring.



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+

Review Comment:
   Try to read back the content of key1 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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(

Review Comment:
   Unrelated to this patch but `getBucketKey()` could be `static`.



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -246,7 +257,7 @@ private boolean deleteSnapshotPath(String snapshotPath,
     return status;
   }
 
-  private void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)
+  public void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)

Review Comment:
   It can be made `private`. I made it `public` because of previous changes. Now that change is modified we can make it `private` again



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,

Review Comment:
   Right. @prashantpogde talked about **disabling** compaction (or any background tasks) on those snapshot DB checkpoints. We agreed it can be done with a `DBOption` when opening those DBs for write.



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,
+                nextDBKey.getKey(), nextDBOmKeyInfo);
+      } else {
+        omMetadataManager.getDeletedTable()
+            .put(nextDBKey.getKey(), nextDBOmKeyInfo);

Review Comment:
   why is this a "put() "while the others above are "putWithBatch()"?



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,
+                nextDBKey.getKey(), nextDBOmKeyInfo);
+      } else {
+        omMetadataManager.getDeletedTable()
+            .put(nextDBKey.getKey(), nextDBOmKeyInfo);

Review Comment:
   why is this a "put()" while the others above are "putWithBatch()"?



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Handles OMSnapshotMoveDeletedKeys Request.
+ */
+public class OMSnapshotMoveDeletedKeysRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotMoveDeletedKeysRequest.class);
+
+  public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+    OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    SnapshotChainManager snapshotChainManager =
+        ozoneManager.getSnapshotChainManager();
+
+    SnapshotMoveDeletedKeysRequest moveDeletedKeysRequest =
+        getOmRequest().getSnapshotMoveDeletedKeysRequest();
+    SnapshotInfo fromSnapshot = SnapshotInfo.getFromProtobuf(
+        moveDeletedKeysRequest.getFromSnapshot());
+
+    // If there is no Non-Deleted Snapshot move the
+    // keys to Active Object Store.
+    SnapshotInfo nextSnapshot = null;
+    OMClientResponse omClientResponse = null;
+    OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    try {
+      nextSnapshot = getNextActiveSnapshot(fromSnapshot,
+          snapshotChainManager, omSnapshotManager);
+
+      // Get next non-deleted snapshot.
+
+      List<SnapshotMoveKeyInfos> activeDBKeysList =
+          moveDeletedKeysRequest.getActiveDBKeysList();
+      List<SnapshotMoveKeyInfos> nextDBKeysList =
+          moveDeletedKeysRequest.getNextDBKeysList();
+
+      OmSnapshot omNextSnapshot = null;
+
+      if (nextSnapshot != null) {
+        omNextSnapshot = (OmSnapshot) omSnapshotManager
+            .checkForSnapshot(nextSnapshot.getVolumeName(),

Review Comment:
   ~~Yeah it can be done. Thanks for the suggestion, will update the patch.~~
   While implementing I noticed, there is no reference for `OMSnapshotManager` in `OMSnapshotMoveDeletedKeysResponse`. I don't think it is a good idea to pass it as a parameter.



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+
+    //Check key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/key2");
+
+    assertNotNull(omKeyInfo1);
+
+  }
+
+  private OmKeyArgs createVolumeBucketKey(String volumeName, String bucketName,
+      BucketLayout bucketLayout, String keyName) throws IOException {
+    // cheat here, just create a volume and bucket entry so that we can
+    // create the keys, we put the same data for key and value since the
+    // system does not decode the object
+    OMRequestTestUtils.addVolumeToOM(omMetadataManager,
+        OmVolumeArgs.newBuilder()
+            .setOwnerName("owner")
+            .setAdminName("admin")
+            .setVolume(volumeName)
+            .build());
+
+    OMRequestTestUtils.addBucketToOM(omMetadataManager,
+        OmBucketInfo.newBuilder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return createKey(volumeName, bucketName, keyName);
+  }
+
+  private OmKeyArgs createKey(String volumeName, String bucketName,
+       String keyName) throws IOException {
+    OmKeyArgs keyArg =
+        new OmKeyArgs.Builder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setAcls(Collections.emptyList())
+            .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+            .setLocationInfoList(new ArrayList<>())
+            .build();
+
+    // Open and write the key without commit it.
+    OpenKeySession session = writeClient.openKey(keyArg);
+    writeClient.commitKey(keyArg, session.getId());
+
+    return keyArg;
+  }
+
+  private void createSnapshot(String volName, String bucketName,
+       String snapName, int count) throws Exception {
+    writeClient.createSnapshot(volName, bucketName, snapName);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return omMetadataManager.getSnapshotInfoTable()
+            .getEstimatedKeyCount() >= count;

Review Comment:
   Wouldn't it be better to use countRowsInTable(): https://github.com/apache/ozone/blob/7bdfbba56f0f9b47d62b702e32ee846a2701b9fe/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L1477



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_TIMEOUT;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME_ONE = "bucket1";
+  private static final String BUCKET_NAME_TWO = "bucket2";
+
+
+  @BeforeAll
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @BeforeEach
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (om != null) {
+      om.stop();
+    }
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    createSnapshotDataForBucket1();
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0);
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME_ONE,
+            getSnapshotPrefix("bucket1snap3"));
+
+    // Check bucket1key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/bucket1key1");
+    Assertions.assertNotNull(omKeyInfo);
+
+    // Check bucket1key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/bucket1key2");
+
+    Assertions.assertNotNull(omKeyInfo1);
+
+  }
+
+  @Test
+  public void testMultipleSnapshotKeyReclaim() throws Exception {
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    int snapshotCount = createSnapshotDataForBucket1();
+
+    OmKeyArgs bucket2key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        BucketLayout.DEFAULT, "bucket2key1");
+
+    OmKeyArgs bucket2key2 = createKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        "bucket2key2");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap1",
+        ++snapshotCount);
+
+    // Both key 1 and key 2 can be reclaimed when Snapshot 1 is deleted.
+    writeClient.deleteKey(bucket2key1);
+    writeClient.deleteKey(bucket2key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap2",
+        ++snapshotCount);
+
+    String snapshotKey2 = "/vol1/bucket2/bucket2snap1";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    Assertions.assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0L);
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    // Check bucket2key1 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key1");
+
+    // Check bucket2key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo2 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key2");
+
+    Assertions.assertNotNull(omKeyInfo1);
+    Assertions.assertNotNull(omKeyInfo2);
+  }
+
+  private OmKeyArgs createVolumeBucketKey(String volumeName, String bucketName,
+      BucketLayout bucketLayout, String keyName) throws IOException {
+    // cheat here, just create a volume and bucket entry so that we can
+    // create the keys, we put the same data for key and value since the
+    // system does not decode the object
+    OMRequestTestUtils.addVolumeToOM(omMetadataManager,
+        OmVolumeArgs.newBuilder()
+            .setOwnerName("owner")
+            .setAdminName("admin")
+            .setVolume(volumeName)
+            .build());
+
+    OMRequestTestUtils.addBucketToOM(omMetadataManager,
+        OmBucketInfo.newBuilder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return createKey(volumeName, bucketName, keyName);
+  }
+
+
+  private int createSnapshotDataForBucket1() throws Exception {
+    int snapshotCount = 0;
+    OmKeyArgs bucket1key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        BucketLayout.DEFAULT, "bucket1key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap1",
+        ++snapshotCount);
+
+    OmKeyArgs bucket1key2 = createKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        "bucket1key2");
+
+    // Key 1 cannot be reclaimed as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(bucket1key1);
+    // Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(bucket1key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap2",
+        ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1key4");
+    OmKeyArgs bucket1key5 = createKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        "bucket1key5");
+    writeClient.deleteKey(bucket1key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap3",
+        ++snapshotCount);
+
+    String snapshotKey2 = "/vol1/bucket1/bucket1snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    Assertions.assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    return snapshotCount;
+  }
+
+  private OmKeyArgs createKey(String volumeName, String bucketName,
+       String keyName) throws IOException {
+    OmKeyArgs keyArg =
+        new OmKeyArgs.Builder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setAcls(Collections.emptyList())
+            .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+            .setLocationInfoList(new ArrayList<>())
+            .build();
+
+    // Open and write the key.
+    OpenKeySession session = writeClient.openKey(keyArg);
+    writeClient.commitKey(keyArg, session.getId());
+
+    return keyArg;
+  }
+
+  private void createSnapshot(String volName, String bucketName,
+       String snapName, int count) throws Exception {
+    writeClient.createSnapshot(volName, bucketName, snapName);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return omMetadataManager.getSnapshotInfoTable()
+            .getEstimatedKeyCount() >= count;

Review Comment:
   Wouldn't it be better to use countRowsInTable(): https://github.com/apache/ozone/blob/7bdfbba56f0f9b47d62b702e32ee846a2701b9fe/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L1477



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_TIMEOUT;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME_ONE = "bucket1";
+  private static final String BUCKET_NAME_TWO = "bucket2";
+
+
+  @BeforeAll
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @BeforeEach
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (om != null) {
+      om.stop();
+    }
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    createSnapshotDataForBucket1();
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0);
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME_ONE,
+            getSnapshotPrefix("bucket1snap3"));
+
+    // Check bucket1key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/bucket1key1");
+    Assertions.assertNotNull(omKeyInfo);
+
+    // Check bucket1key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/bucket1key2");
+
+    Assertions.assertNotNull(omKeyInfo1);
+
+  }
+
+  @Test
+  public void testMultipleSnapshotKeyReclaim() throws Exception {
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    int snapshotCount = createSnapshotDataForBucket1();
+
+    OmKeyArgs bucket2key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        BucketLayout.DEFAULT, "bucket2key1");
+
+    OmKeyArgs bucket2key2 = createKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        "bucket2key2");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap1",
+        ++snapshotCount);
+
+    // Both key 1 and key 2 can be reclaimed when Snapshot 1 is deleted.
+    writeClient.deleteKey(bucket2key1);
+    writeClient.deleteKey(bucket2key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap2",
+        ++snapshotCount);
+
+    String snapshotKey2 = "/vol1/bucket2/bucket2snap1";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    Assertions.assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0L);
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    // Check bucket2key1 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key1");
+
+    // Check bucket2key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo2 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key2");
+
+    Assertions.assertNotNull(omKeyInfo1);
+    Assertions.assertNotNull(omKeyInfo2);
+  }
+
+  private OmKeyArgs createVolumeBucketKey(String volumeName, String bucketName,
+      BucketLayout bucketLayout, String keyName) throws IOException {
+    // cheat here, just create a volume and bucket entry so that we can
+    // create the keys, we put the same data for key and value since the
+    // system does not decode the object
+    OMRequestTestUtils.addVolumeToOM(omMetadataManager,
+        OmVolumeArgs.newBuilder()
+            .setOwnerName("owner")
+            .setAdminName("admin")
+            .setVolume(volumeName)
+            .build());
+
+    OMRequestTestUtils.addBucketToOM(omMetadataManager,
+        OmBucketInfo.newBuilder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return createKey(volumeName, bucketName, keyName);
+  }
+
+
+  private int createSnapshotDataForBucket1() throws Exception {
+    int snapshotCount = 0;
+    OmKeyArgs bucket1key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        BucketLayout.DEFAULT, "bucket1key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap1",
+        ++snapshotCount);
+
+    OmKeyArgs bucket1key2 = createKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        "bucket1key2");
+
+    // Key 1 cannot be reclaimed as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(bucket1key1);
+    // Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(bucket1key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap2",
+        ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1key4");
+    OmKeyArgs bucket1key5 = createKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        "bucket1key5");
+    writeClient.deleteKey(bucket1key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap3",
+        ++snapshotCount);
+
+    String snapshotKey2 = "/vol1/bucket1/bucket1snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    Assertions.assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    return snapshotCount;
+  }
+
+  private OmKeyArgs createKey(String volumeName, String bucketName,
+       String keyName) throws IOException {
+    OmKeyArgs keyArg =
+        new OmKeyArgs.Builder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setAcls(Collections.emptyList())
+            .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+            .setLocationInfoList(new ArrayList<>())
+            .build();
+
+    // Open and write the key.
+    OpenKeySession session = writeClient.openKey(keyArg);
+    writeClient.commitKey(keyArg, session.getId());
+
+    return keyArg;
+  }
+
+  private void createSnapshot(String volName, String bucketName,
+       String snapName, int count) throws Exception {
+    writeClient.createSnapshot(volName, bucketName, snapName);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return omMetadataManager.getSnapshotInfoTable()
+            .getEstimatedKeyCount() >= count;

Review Comment:
   Since getEstimatedCount() is "sometimes inaccurate", wouldn't it be better to use countRowsInTable(): https://github.com/apache/ozone/blob/7bdfbba56f0f9b47d62b702e32ee846a2701b9fe/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L1477 



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+
+    //Check key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/key2");
+
+    assertNotNull(omKeyInfo1);
+
+  }
+
+  private OmKeyArgs createVolumeBucketKey(String volumeName, String bucketName,
+      BucketLayout bucketLayout, String keyName) throws IOException {
+    // cheat here, just create a volume and bucket entry so that we can
+    // create the keys, we put the same data for key and value since the
+    // system does not decode the object
+    OMRequestTestUtils.addVolumeToOM(omMetadataManager,
+        OmVolumeArgs.newBuilder()
+            .setOwnerName("owner")
+            .setAdminName("admin")
+            .setVolume(volumeName)
+            .build());
+
+    OMRequestTestUtils.addBucketToOM(omMetadataManager,
+        OmBucketInfo.newBuilder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return createKey(volumeName, bucketName, keyName);
+  }
+
+  private OmKeyArgs createKey(String volumeName, String bucketName,
+       String keyName) throws IOException {
+    OmKeyArgs keyArg =
+        new OmKeyArgs.Builder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setAcls(Collections.emptyList())
+            .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+            .setLocationInfoList(new ArrayList<>())
+            .build();
+
+    // Open and write the key without commit it.
+    OpenKeySession session = writeClient.openKey(keyArg);
+    writeClient.commitKey(keyArg, session.getId());
+
+    return keyArg;
+  }
+
+  private void createSnapshot(String volName, String bucketName,
+       String snapName, int count) throws Exception {
+    writeClient.createSnapshot(volName, bucketName, snapName);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return omMetadataManager.getSnapshotInfoTable()
+            .getEstimatedKeyCount() >= count;

Review Comment:
   Wouldn't it be better to use countRowsInTable(): https://github.com/apache/ozone/blob/7bdfbba56f0f9b47d62b702e32ee846a2701b9fe/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java#L1477



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1701,6 +1704,18 @@ message DeleteSnapshotRequest {
   optional uint64 deletionTime = 4;
 }
 
+message SnapshotMoveDeletedKeysRequest {
+  optional SnapshotInfo fromSnapshot = 1;
+  optional SnapshotInfo nextSnapshot = 2;
+  repeated SnapshotMoveKeyInfos activeDBKeys = 3;
+  repeated SnapshotMoveKeyInfos nextDBKeys = 4;

Review Comment:
   `activeDBKeys` are keys that should be moved to active db's `deletedTable`. `nextDBKeys` are keys that should be moved to nextSnapshot's `deletedTable`



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConsts.java:
##########
@@ -237,6 +237,8 @@ public enum Units { TB, GB, MB, KB, B }
 
   public static final int INVALID_PORT = -1;
 
+  public static final long OBJECT_ID_DEFAULT = 0L;

Review Comment:
   Make the variable name less generic. Maybe `OBJECT_ID_RECLAIM_BLOCKS`
   
   And put a comment there to clearly describe the intention of the constant.



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              KeyValuePair.Builder toActiveDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+              KeyValuePair.Builder toNextDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, nextSnapshot,
+                toActiveDBList, toNextDBList);
+            snapshotLimit--;
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+
+        successRunCount.incrementAndGet();
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(KeyValuePair.Builder toActiveDb,
+        KeyValuePair.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        SnapshotInfo nextSnapshot, List<KeyValuePair> toActiveDBList,
+        List<KeyValuePair> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      if (nextSnapshot != null) {
+        moveDeletedKeysBuilder.setNextSnapshot(nextSnapshot.getProtobuf());
+      }
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      if (chainManager.hasPreviousPathSnapshot(snapInfo.getSnapshotPath(),
+          snapInfo.getSnapshotID())) {
+        String previousPathSnapshot = chainManager.previousPathSnapshot(
+            snapInfo.getSnapshotPath(), snapInfo.getSnapshotID());
+        String tableKey = chainManager.getTableKey(previousPathSnapshot);
+        return omSnapshotManager.getSnapshotInfo(tableKey);
+      }
+      return null;
+    }
+
+    /**
+     * Get the next non deleted snapshot in the snapshot chain.
+     */
+    private SnapshotInfo getNextNonDeletedSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      while (chainManager.hasNextPathSnapshot(snapInfo.getSnapshotPath(), 
+          snapInfo.getSnapshotID())) {
+
+        String nextPathSnapshot =
+            chainManager.nextPathSnapshot(
+                snapInfo.getSnapshotPath(), snapInfo.getSnapshotID());
+
+        String tableKey = chainManager.getTableKey(nextPathSnapshot);
+        SnapshotInfo nextSnapshotInfo =
+            omSnapshotManager.getSnapshotInfo(tableKey);
+
+        if (nextSnapshotInfo.getSnapshotStatus().equals(
+            SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE)) {
+          return nextSnapshotInfo;
+        }
+      }
+      return null;
+    }
+
+    private void submitRequest(OMRequest omRequest) {
+      try {
+        if (isRatisEnabled()) {
+          OzoneManagerRatisServer server = ozoneManager.getOmRatisServer();
+
+          RaftClientRequest raftClientRequest = RaftClientRequest.newBuilder()
+              .setClientId(clientId)
+              .setServerId(server.getRaftPeerId())
+              .setGroupId(server.getRaftGroupId())
+              .setCallId(runCount.get())
+              .setMessage(Message.valueOf(
+                  OMRatisHelper.convertRequestToByteString(omRequest)))
+              .setType(RaftClientRequest.writeRequestType())
+              .build();
+
+          server.submitRequest(omRequest, raftClientRequest);
+        } else {
+          ozoneManager.getOmServerProtocol().submitRequest(null, omRequest);
+        }
+      } catch (ServiceException e) {
+        LOG.error("Snapshot Deleting request failed. " +
+            "Will retry at next run.", e);
+      }
+    }
+
+    private boolean isRatisEnabled() {
+      return ozoneManager.isRatisEnabled();
+    }
+
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+    queue.add(new SnapshotDeletingTask());
+    return queue;
+  }
+
+  private boolean shouldRun() {
+    return !suspended.get() && ozoneManager.isLeaderReady();
+  }
+
+  /**
+   * Suspend the service (for testing).
+   */
+  @VisibleForTesting

Review Comment:
   1. Right now this is only for testing purposes. Can be extended later for further usage. 
   2. Noted, will change. 
   3. 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] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>

Review Comment:
   Finishing this up here [HDDS-7883](https://issues.apache.org/jira/browse/HDDS-7883)



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+
+    //Check key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/key2");
+
+    assertNotNull(omKeyInfo1);
+
+  }
+
+  private OmKeyArgs createVolumeBucketKey(String volumeName, String bucketName,
+      BucketLayout bucketLayout, String keyName) throws IOException {
+    // cheat here, just create a volume and bucket entry so that we can
+    // create the keys, we put the same data for key and value since the
+    // system does not decode the object
+    OMRequestTestUtils.addVolumeToOM(omMetadataManager,
+        OmVolumeArgs.newBuilder()
+            .setOwnerName("owner")
+            .setAdminName("admin")
+            .setVolume(volumeName)
+            .build());
+
+    OMRequestTestUtils.addBucketToOM(omMetadataManager,
+        OmBucketInfo.newBuilder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return createKey(volumeName, bucketName, keyName);
+  }
+
+  private OmKeyArgs createKey(String volumeName, String bucketName,
+       String keyName) throws IOException {
+    OmKeyArgs keyArg =
+        new OmKeyArgs.Builder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setAcls(Collections.emptyList())
+            .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+            .setLocationInfoList(new ArrayList<>())
+            .build();
+
+    // Open and write the key without commit it.
+    OpenKeySession session = writeClient.openKey(keyArg);
+    writeClient.commitKey(keyArg, session.getId());

Review Comment:
   @DaveTeng0 nice catch



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,

Review Comment:
   Right. @prashantpogde talked about **disabling** compaction (~~or any background tasks~~ flush would still need to happen) on those snapshot DB checkpoints. We agreed it can be done with a `DBOption` when opening those DBs for write.



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

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

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


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


[GitHub] [ozone] prashantpogde commented on pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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

   We could also create an instance of Keydeletion/DirectoryDeletionService service that could operate on a snapshot/active OS instance. That way we can create as many instances as needed to scale.


-- 
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] umamaheswararao commented on pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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

   @hemantk-12 @sumitagrawl 


-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {

Review Comment:
   +1 @GeorgeJahad 's comment that we need to start at the first key in the bucket scope.
   
   And break the while loop once out of the bucket scope.



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+

Review Comment:
   Try to read back the content of key1 as well, to ensure the blocks aren't reclaimed.



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()

Review Comment:
   @sumitagrawl Theoretically it shouldn't because we prevent buckets deletion when they still have any snapshots. But yes we should handle the case (throw exception).



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setPathPreviousSnapshotID(latestPathSnapshot);
+      }
+
+      if (latestGlobalSnapshot == null || latestGlobalSnapshot.isEmpty()) {
+        snapshotInfo.setGlobalPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setGlobalPreviousSnapshotID(latestGlobalSnapshot);
+      }
+
+      snapshotChainManager.addSnapshot(snapshotInfo);

Review Comment:
   CMIIW but this seems to be the only place where snapshots are added to OM's `SnapshotChainManager`?
   
   Since `SnapshotChainManager` persists its entire state in-memory, we will have to reconstruct the snapshot chain on OM startup (reconstruct from scratch first, persist it later if too time consuming).



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {

Review Comment:
   I'm not familiar with "getEstimatedKeyCount()".  Since it is "estimated" is there any chance that this will cause us to skip keys that should be gc'd?



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

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

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


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


[GitHub] [ozone] smengcl commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot fromSnapshot;
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+  private List<SnapshotMoveKeyInfos> reclaimKeyList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       @Nonnull OmSnapshot omFromSnapshot, OmSnapshot omNextSnapshot,
+       List<SnapshotMoveKeyInfos> nextDBKeysList,
+       List<SnapshotMoveKeyInfos> reclaimKeysList) {
+    super(omResponse);
+    this.fromSnapshot = omFromSnapshot;
+    this.nextSnapshot = omNextSnapshot;
+    this.nextDBKeysList = nextDBKeysList;
+    this.reclaimKeyList = reclaimKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    if (nextSnapshot != null) {
+      DBStore nextSnapshotStore = nextSnapshot.getMetadataManager().getStore();
+      // Init Batch Operation for snapshot db.
+      try (BatchOperation writeBatch = nextSnapshotStore.initBatchOperation()) {
+        processKeys(writeBatch, nextSnapshot.getMetadataManager(),
+            nextDBKeysList);
+        nextSnapshotStore.commitBatchOperation(writeBatch);
+      }
+    } else {
+      // Handle case when there is no next Snapshot

Review Comment:
   nit
   
   ```suggestion
         // Handle the case where there is no next Snapshot
   ```



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_TIMEOUT;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME_ONE = "bucket1";
+  private static final String BUCKET_NAME_TWO = "bucket2";
+
+
+  @BeforeAll
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @BeforeEach
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (om != null) {
+      om.stop();
+    }
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    createSnapshotDataForBucket1();
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0);
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME_ONE,
+            getSnapshotPrefix("bucket1snap3"));
+
+    // Check bucket1key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/bucket1key1");
+    Assertions.assertNotNull(omKeyInfo);
+
+    // Check bucket1key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/bucket1key2");
+
+    Assertions.assertNotNull(omKeyInfo1);
+
+  }
+
+  @Test
+  public void testMultipleSnapshotKeyReclaim() throws Exception {
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    int snapshotCount = createSnapshotDataForBucket1();
+
+    OmKeyArgs bucket2key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        BucketLayout.DEFAULT, "bucket2key1");
+
+    OmKeyArgs bucket2key2 = createKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        "bucket2key2");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap1",
+        ++snapshotCount);
+
+    // Both key 1 and key 2 can be reclaimed when Snapshot 1 is deleted.
+    writeClient.deleteKey(bucket2key1);
+    writeClient.deleteKey(bucket2key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap2",
+        ++snapshotCount);
+
+    String snapshotKey2 = "/vol1/bucket2/bucket2snap1";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    Assertions.assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0L);
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    // Check bucket2key1 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key1");
+
+    // Check bucket2key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo2 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key2");
+
+    Assertions.assertNotNull(omKeyInfo1);

Review Comment:
   I don't think this test is actually working properly; it continues to pass if you comment out the following lines:
   
       //snapshotDeletingService.resume();
       //snapshotDeletingService.setSuccessRunCount(0L);
       //GenericTestUtils.waitFor(() ->
       //        snapshotDeletingService.getSuccessfulRunCount() >= 1,
       //    1000, 10000);
   
   I think it is only passing because @smengcl 's pr hasn't been merged yet: https://github.com/apache/ozone/pull/4280  Once that is merged in I believe it will fail, because I don't think it is actually testing the correct thing.  I believe the keys will only be moved to the active fs when bucket2snap2 is deleted.  What do 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] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1701,6 +1704,18 @@ message DeleteSnapshotRequest {
   optional uint64 deletionTime = 4;
 }
 
+message SnapshotMoveDeletedKeysRequest {
+  optional SnapshotInfo fromSnapshot = 1;
+  optional SnapshotInfo nextSnapshot = 2;

Review Comment:
   It is being used in `OMSnapshotMoveDeletedKeysResponse`



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

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

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


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


[GitHub] [ozone] prashantpogde commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,

Review Comment:
   There should be no compaction running on the Snapshots.



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();

Review Comment:
   seek to first key for this bucket?



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+
+    //Check key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/key2");
+
+    assertNotNull(omKeyInfo1);
+
+  }

Review Comment:
   Relevant UT for reference: https://github.com/apache/ozone/pull/4273/files#diff-71c1f01e984fc77a5d653be56ec27d6347b2f8104c717a936b6a53a25c5116e3R602-R627



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/SnapshotInfo.java:
##########
@@ -499,8 +513,9 @@ public boolean equals(Object o) {
         name.equals(that.name) && volumeName.equals(that.volumeName) &&
         bucketName.equals(that.bucketName) &&
         snapshotStatus == that.snapshotStatus &&
-        pathPreviousSnapshotID.equals(that.pathPreviousSnapshotID) &&
-        globalPreviousSnapshotID.equals(that.globalPreviousSnapshotID) &&
+        Objects.equals(pathPreviousSnapshotID, that.pathPreviousSnapshotID) &&
+        Objects.equals(
+            globalPreviousSnapshotID, that.globalPreviousSnapshotID) &&

Review Comment:
   nice. avoids NPE



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3155,6 +3155,14 @@
       sst filtering service per time interval.
     </description>
   </property>
+  <property>
+    <name>ozone.snapshot.deleting.limit.per.task</name>

Review Comment:
   consistency
   
   ```suggestion
       <name>ozone.snapshot.deletion.limit.per.task</name>
   ```



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java:
##########
@@ -837,6 +838,7 @@ protected OmKeyInfo wrapUncommittedBlocksAsPseudoKey(
     LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
         uncommitted, omKeyInfo.getKeyName());
     OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
+    pseudoKeyInfo.setObjectID(OBJECT_ID_DEFAULT);

Review Comment:
   Let's put a comment above this line as well that we are using this as a special marker to indicate that this key's blocks can be unconditionally reclaimed in `SnapshotDeletingService`.



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java:
##########
@@ -234,6 +234,16 @@ public final class OzoneConfigKeys {
       OZONE_SNAPSHOT_SST_FILTERING_SERVICE_TIMEOUT_DEFAULT = "300s";
       // 300s for default
 
+  public static final String OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL =
+      "ozone.snapshot.deletion.service.interval";
+  public static final String
+      OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL_DEFAULT = "30s";
+
+  public static final String OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT =
+      "ozone.snapshot.deletion.service.timeout";
+  public static final String
+      OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT_DEFAULT = "300s";

Review Comment:
   Just a note that this is not a hard time-out that would stop task execution. AFAIK similar to other tasks this would only [print out a warning](https://github.com/apache/ozone/blob/a66e43e2838f087229cb4b0fac1e1dc969ab80dd/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BackgroundService.java#L129-L132) if the task run exceed the given time.



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

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

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


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


[GitHub] [ozone] smengcl commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1691,6 +1694,18 @@ message DeleteSnapshotRequest {
   optional uint64 deletionTime = 4;
 }
 
+message SnapshotMoveDeletedKeysRequest {
+  required SnapshotInfo fromSnapshot = 1;

Review Comment:
   ```suggestion
     optional SnapshotInfo fromSnapshot = 1;
   ```



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -131,6 +133,7 @@ private void addSnapshotPath(String snapshotPath,
             new SnapshotChainInfo(snapshotID, prevPathID, null));
 
     // set state variable latestPath snapshot entry to this snapshotID
+    snapshotPathToKey.put(snapshotID, snapTableKey);
     latestPathSnapshotID.put(snapshotPath, snapshotID);

Review Comment:
   Comment out of order.
   
   ```suggestion
       // store snapshot ID to snapshot DB table key in the map
       snapshotPathToKey.put(snapshotID, snapTableKey);
       // set state variable latestPath snapshot entry to this snapshotID
       latestPathSnapshotID.put(snapshotPath, snapshotID);
   ```



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {

Review Comment:
   `snapshotDeletedTable.isEmpty()` should be of better use here, as `snapshotDeletedTable.getEstimatedKeyCount()` can be inaccurate. Updating the 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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,

Review Comment:
   Is it a problem that we are doing the put with the snapshots metadata manager, but the commit is happening to the omMetadataManage?: https://github.com/apache/ozone/blob/3de525c60c40441a938f84ffda61ebd2b8bad50b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerDoubleBuffer.java#L346-L347



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super(SnapshotDeletingService.class.getSimpleName(), interval,
+        TimeUnit.MILLISECONDS, SNAPSHOT_DELETING_CORE_POOL_SIZE,
+        serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.isEmpty()) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+          if (bucketInfo == null) {
+            throw new OMException("Bucket " + snapInfo.getBucketName() +
+                " is not found", BUCKET_NOT_FOUND);
+          }
+
+          //TODO: [SNAPSHOT] Add lock to deletedTable and Active DB.
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<SnapshotMoveKeyInfos> toActiveDBList = new ArrayList<>();
+          List<SnapshotMoveKeyInfos> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            String snapshotBucketKey = dbBucketKey + OzoneConsts.OM_KEY_PREFIX;
+            iterator.seek(snapshotBucketKey);
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              String deletedKey = deletedKeyValue.getKey();
+
+              // Exit if it is out of the bucket scope.
+              if (!deletedKey.contains(snapshotBucketKey)) {
+                break;
+              }
+
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              SnapshotMoveKeyInfos.Builder toActiveDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+              SnapshotMoveKeyInfos.Builder toNextDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, toActiveDBList,
+                toNextDBList);
+            snapshotLimit--;
+            successRunCount.incrementAndGet();
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(SnapshotMoveKeyInfos.Builder toActiveDb,
+        SnapshotMoveKeyInfos.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        List<SnapshotMoveKeyInfos> toActiveDBList,
+        List<SnapshotMoveKeyInfos> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: [SNAPSHOT] Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)

Review Comment:
   Should this method only return active snapshots?



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>

Review Comment:
   Yes we can do 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] smengcl commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              KeyValuePair.Builder toActiveDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+              KeyValuePair.Builder toNextDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, nextSnapshot,
+                toActiveDBList, toNextDBList);
+            snapshotLimit--;
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+
+        successRunCount.incrementAndGet();
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(KeyValuePair.Builder toActiveDb,
+        KeyValuePair.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        SnapshotInfo nextSnapshot, List<KeyValuePair> toActiveDBList,
+        List<KeyValuePair> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      if (nextSnapshot != null) {
+        moveDeletedKeysBuilder.setNextSnapshot(nextSnapshot.getProtobuf());
+      }
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      if (chainManager.hasPreviousPathSnapshot(snapInfo.getSnapshotPath(),
+          snapInfo.getSnapshotID())) {
+        String previousPathSnapshot = chainManager.previousPathSnapshot(
+            snapInfo.getSnapshotPath(), snapInfo.getSnapshotID());
+        String tableKey = chainManager.getTableKey(previousPathSnapshot);
+        return omSnapshotManager.getSnapshotInfo(tableKey);
+      }
+      return null;
+    }
+
+    /**
+     * Get the next non deleted snapshot in the snapshot chain.
+     */
+    private SnapshotInfo getNextNonDeletedSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      while (chainManager.hasNextPathSnapshot(snapInfo.getSnapshotPath(), 
+          snapInfo.getSnapshotID())) {
+
+        String nextPathSnapshot =
+            chainManager.nextPathSnapshot(
+                snapInfo.getSnapshotPath(), snapInfo.getSnapshotID());
+
+        String tableKey = chainManager.getTableKey(nextPathSnapshot);
+        SnapshotInfo nextSnapshotInfo =
+            omSnapshotManager.getSnapshotInfo(tableKey);
+
+        if (nextSnapshotInfo.getSnapshotStatus().equals(
+            SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE)) {
+          return nextSnapshotInfo;

Review Comment:
   Judging from this, shouldn't this helper method be called `getNextActiveSnapshot` instead?
   
   `NonDeleted` would mean either `ACTIVE` or `RECLAIMED`.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              KeyValuePair.Builder toActiveDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+              KeyValuePair.Builder toNextDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, nextSnapshot,
+                toActiveDBList, toNextDBList);
+            snapshotLimit--;
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+
+        successRunCount.incrementAndGet();
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(KeyValuePair.Builder toActiveDb,
+        KeyValuePair.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        SnapshotInfo nextSnapshot, List<KeyValuePair> toActiveDBList,
+        List<KeyValuePair> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      if (nextSnapshot != null) {
+        moveDeletedKeysBuilder.setNextSnapshot(nextSnapshot.getProtobuf());
+      }
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      if (chainManager.hasPreviousPathSnapshot(snapInfo.getSnapshotPath(),
+          snapInfo.getSnapshotID())) {
+        String previousPathSnapshot = chainManager.previousPathSnapshot(
+            snapInfo.getSnapshotPath(), snapInfo.getSnapshotID());
+        String tableKey = chainManager.getTableKey(previousPathSnapshot);
+        return omSnapshotManager.getSnapshotInfo(tableKey);
+      }
+      return null;
+    }
+
+    /**
+     * Get the next non deleted snapshot in the snapshot chain.
+     */
+    private SnapshotInfo getNextNonDeletedSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      while (chainManager.hasNextPathSnapshot(snapInfo.getSnapshotPath(), 
+          snapInfo.getSnapshotID())) {

Review Comment:
   Not required in this PR, but for cleanliness we could implement an iterator in `chainManager`. e.g.:
   
   ```java
   SnapshotChainIterator it = chainManager.getIteratorForPath(snapshotPath);
   it.seek(snapshotId);
   ```
   
   or to get the global snapshot chain iterator:
   
   ```java
   SnapshotChainIterator it = chainManager.getIterator();
   ```
   
   with which we would be able to move the iterator forward or backward:
   
   ```
   while (it.hasNext()) {
     SnapshotChainInfo nextPathSnapshot = it.next();
     String snapshotId = nextPathSnapshot.getSnapshotID();
   }
   ```
   
   or
   
   ```
   it.hasPrev();
   SnapshotChainInfo prevPathSnapshot = it.prev();
   ```



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              KeyValuePair.Builder toActiveDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+              KeyValuePair.Builder toNextDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, nextSnapshot,
+                toActiveDBList, toNextDBList);
+            snapshotLimit--;
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+
+        successRunCount.incrementAndGet();
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(KeyValuePair.Builder toActiveDb,
+        KeyValuePair.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        SnapshotInfo nextSnapshot, List<KeyValuePair> toActiveDBList,
+        List<KeyValuePair> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      if (nextSnapshot != null) {
+        moveDeletedKeysBuilder.setNextSnapshot(nextSnapshot.getProtobuf());
+      }
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: Handle Renamed Keys

Review Comment:
   To be done in HDDS-7984. Detailed in the design doc addition.



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);

Review Comment:
   We might need one or two mutexes (one for the snapshot's `deletedTable`, maybe another one for active DB's) to protect the logic from this method call all the way down to `submitSnapshotMoveDeletedKeys()` below to prevent others from operating on those tables at the same time (e.g. `KeyDeletingTask` when GC is fully implemented).
   
   You may add a `TODO` here for now. I have implemented a rudimentary lock for active DB's table in my PR draft https://github.com/apache/ozone/pull/4280/files#diff-d4e4dbf0e95db46eded21d40936d1b18cd738dc19fa10ec45dc88211b4b498e3R269-R282 .
   
   Also need to check whether snapshot chain handles snapshot updates (e.g. when unlinking a snapshot after it has been reclaimed) correctly.



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              KeyValuePair.Builder toActiveDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+              KeyValuePair.Builder toNextDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, nextSnapshot,
+                toActiveDBList, toNextDBList);
+            snapshotLimit--;
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+
+        successRunCount.incrementAndGet();
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(KeyValuePair.Builder toActiveDb,
+        KeyValuePair.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        SnapshotInfo nextSnapshot, List<KeyValuePair> toActiveDBList,
+        List<KeyValuePair> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      if (nextSnapshot != null) {
+        moveDeletedKeysBuilder.setNextSnapshot(nextSnapshot.getProtobuf());
+      }
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)

Review Comment:
   Doesn't this method need to be bucket specfic?  See section 4.1.b.a of the design doc:
   
   https://www.notion.so/fossil-i/Design-Ozone-Snapshot-Deletion-Garbage-Collection-based-on-key-deletedTable-2a624480dc7c4bc3ad608cbf86a25541?pvs=4#cd47388bf64142809592ebe544f64aa0



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              KeyValuePair.Builder toActiveDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+              KeyValuePair.Builder toNextDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, nextSnapshot,
+                toActiveDBList, toNextDBList);
+            snapshotLimit--;
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+
+        successRunCount.incrementAndGet();
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(KeyValuePair.Builder toActiveDb,
+        KeyValuePair.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        SnapshotInfo nextSnapshot, List<KeyValuePair> toActiveDBList,
+        List<KeyValuePair> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      if (nextSnapshot != null) {
+        moveDeletedKeysBuilder.setNextSnapshot(nextSnapshot.getProtobuf());
+      }
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      if (chainManager.hasPreviousPathSnapshot(snapInfo.getSnapshotPath(),
+          snapInfo.getSnapshotID())) {
+        String previousPathSnapshot = chainManager.previousPathSnapshot(
+            snapInfo.getSnapshotPath(), snapInfo.getSnapshotID());
+        String tableKey = chainManager.getTableKey(previousPathSnapshot);
+        return omSnapshotManager.getSnapshotInfo(tableKey);
+      }
+      return null;
+    }
+
+    /**
+     * Get the next non deleted snapshot in the snapshot chain.
+     */
+    private SnapshotInfo getNextNonDeletedSnapshot(SnapshotInfo snapInfo)

Review Comment:
   same as above



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,
+                nextDBKey.getKey(), nextDBOmKeyInfo);
+      } else {
+        omMetadataManager.getDeletedTable()
+            .put(nextDBKey.getKey(), nextDBOmKeyInfo);

Review Comment:
   Nice catch. Should be `putWithBatch()`



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Handles OMSnapshotMoveDeletedKeys Request.
+ */
+public class OMSnapshotMoveDeletedKeysRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotMoveDeletedKeysRequest.class);
+
+  public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+    OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    SnapshotChainManager snapshotChainManager =
+        ozoneManager.getSnapshotChainManager();
+
+    SnapshotMoveDeletedKeysRequest moveDeletedKeysRequest =
+        getOmRequest().getSnapshotMoveDeletedKeysRequest();
+    SnapshotInfo fromSnapshot = SnapshotInfo.getFromProtobuf(
+        moveDeletedKeysRequest.getFromSnapshot());
+
+    // If there is no Non-Deleted Snapshot move the
+    // keys to Active Object Store.
+    SnapshotInfo nextSnapshot = null;
+    OMClientResponse omClientResponse = null;
+    OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    try {
+      nextSnapshot = getNextActiveSnapshot(fromSnapshot,
+          snapshotChainManager, omSnapshotManager);
+
+      // Get next non-deleted snapshot.
+
+      List<SnapshotMoveKeyInfos> activeDBKeysList =
+          moveDeletedKeysRequest.getActiveDBKeysList();
+      List<SnapshotMoveKeyInfos> nextDBKeysList =
+          moveDeletedKeysRequest.getNextDBKeysList();
+
+      OmSnapshot omNextSnapshot = null;
+
+      if (nextSnapshot != null) {
+        omNextSnapshot = (OmSnapshot) omSnapshotManager
+            .checkForSnapshot(nextSnapshot.getVolumeName(),

Review Comment:
   I think it would better to do this call from within OMSnapshotMoveDeletedKeysResponse.  You could just pass in the SnapshotInfo parameter instead of the OMSnapshot parameter.  What do 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] smengcl commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setPathPreviousSnapshotID(latestPathSnapshot);
+      }
+
+      if (latestGlobalSnapshot == null || latestGlobalSnapshot.isEmpty()) {
+        snapshotInfo.setGlobalPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setGlobalPreviousSnapshotID(latestGlobalSnapshot);
+      }
+
+      snapshotChainManager.addSnapshot(snapshotInfo);

Review Comment:
   CMIIW but this seems to be the only place where snapshots are added to OM's `SnapshotChainManager`?
   
   Since `SnapshotChainManager` persists its entire state in-memory, we will have to reconstruct the snapshot chain during OM startup (first implementation could reconstruct it from `snapshotInfoTable`. persist it later if too time consuming).



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setPathPreviousSnapshotID(latestPathSnapshot);
+      }
+
+      if (latestGlobalSnapshot == null || latestGlobalSnapshot.isEmpty()) {
+        snapshotInfo.setGlobalPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setGlobalPreviousSnapshotID(latestGlobalSnapshot);
+      }
+
+      snapshotChainManager.addSnapshot(snapshotInfo);

Review Comment:
   Filed HDDS-7970 . cc @neils-dev 



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,
+                nextDBKey.getKey(), nextDBOmKeyInfo);
+      } else {
+        omMetadataManager.getDeletedTable().putWithBatch(
+            batchOperation, nextDBKey.getKey(), nextDBOmKeyInfo);
+      }
+    }
+  }
+
+  private RepeatedOmKeyInfo createRepeatedOmKeyInfo(List<KeyInfo> keyInfoList)
+      throws IOException {
+    RepeatedOmKeyInfo result = null;
+
+    for (KeyInfo keyInfo: keyInfoList) {
+      if (result == null) {
+        result = new RepeatedOmKeyInfo(OmKeyInfo.getFromProtobuf(keyInfo));
+      } else {
+        result.addOmKeyInfo(OmKeyInfo.getFromProtobuf(keyInfo));
+      }
+    }
+
+    return result;
+  }
+}

Review Comment:
   nit
   ```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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/SnapshotInfo.java:
##########
@@ -368,10 +368,17 @@ public OzoneManagerProtocolProtos.SnapshotInfo getProtobuf() {
         .setBucketName(bucketName)
         .setSnapshotStatus(snapshotStatus.toProto())
         .setCreationTime(creationTime)
-        .setDeletionTime(deletionTime)
-        .setPathPreviousSnapshotID(pathPreviousSnapshotID)
-        .setGlobalPreviousSnapshotID(globalPreviousSnapshotID)
-        .setSnapshotPath(snapshotPath)
+        .setDeletionTime(deletionTime);
+
+    if (pathPreviousSnapshotID != null) {
+      sib.setPathPreviousSnapshotID(pathPreviousSnapshotID);
+    }
+
+    if (globalPreviousSnapshotID != null) {
+      sib.setGlobalPreviousSnapshotID(globalPreviousSnapshotID);
+    }
+
+    sib.setSnapshotPath(snapshotPath)
         .setCheckpointDir(checkpointDir)
         .setDbTxSequenceNumber(dbTxSequenceNumber);

Review Comment:
   alignment
   
   ```suggestion
          .setCheckpointDir(checkpointDir)
          .setDbTxSequenceNumber(dbTxSequenceNumber);
   ```



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.

Review Comment:
   nit
   ```suggestion
       // Key 1 cannot be reclaimed as it is still referenced by Snapshot 1.
   ```



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();

Review Comment:
   In raw use of RocksDB iterator API this is necessary. However, our wrapper already does a seek here so it is no longer necessary: https://github.com/apache/ozone/blob/192eff11a9cc02ccad7d3b4aa58126a71da801e6/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RDBStoreIterator.java#L62
   
   ```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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>

Review Comment:
   @sumitagrawl Yes we do need that. That can be made a configurable value.



-- 
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] sumitagrawl commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>

Review Comment:
   Do we need set limit for number of deletedKeys? similar to keyDeletingService



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager

Review Comment:
   For comparision of keys across snapshot, there is a possibility of volume/bucket removed, or recreated, the objectId is changed. So we can not use volume, and bucket name, and also key name can not be used. Need to use objectId additionally.
   Please recheck this part for this ...



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()

Review Comment:
   If bucket is removed in active db, this might be null in this case.



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              KeyValuePair.Builder toActiveDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+              KeyValuePair.Builder toNextDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, nextSnapshot,
+                toActiveDBList, toNextDBList);
+            snapshotLimit--;
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+
+        successRunCount.incrementAndGet();
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(KeyValuePair.Builder toActiveDb,
+        KeyValuePair.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        SnapshotInfo nextSnapshot, List<KeyValuePair> toActiveDBList,
+        List<KeyValuePair> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      if (nextSnapshot != null) {
+        moveDeletedKeysBuilder.setNextSnapshot(nextSnapshot.getProtobuf());
+      }
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      if (chainManager.hasPreviousPathSnapshot(snapInfo.getSnapshotPath(),
+          snapInfo.getSnapshotID())) {
+        String previousPathSnapshot = chainManager.previousPathSnapshot(
+            snapInfo.getSnapshotPath(), snapInfo.getSnapshotID());
+        String tableKey = chainManager.getTableKey(previousPathSnapshot);
+        return omSnapshotManager.getSnapshotInfo(tableKey);
+      }
+      return null;
+    }
+
+    /**
+     * Get the next non deleted snapshot in the snapshot chain.
+     */
+    private SnapshotInfo getNextNonDeletedSnapshot(SnapshotInfo snapInfo)

Review Comment:
   same as above



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              KeyValuePair.Builder toActiveDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+              KeyValuePair.Builder toNextDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, nextSnapshot,
+                toActiveDBList, toNextDBList);
+            snapshotLimit--;
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+
+        successRunCount.incrementAndGet();
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(KeyValuePair.Builder toActiveDb,
+        KeyValuePair.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        SnapshotInfo nextSnapshot, List<KeyValuePair> toActiveDBList,
+        List<KeyValuePair> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      if (nextSnapshot != null) {
+        moveDeletedKeysBuilder.setNextSnapshot(nextSnapshot.getProtobuf());
+      }
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)

Review Comment:
   sounds good, 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] smengcl commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {

Review Comment:
   Resolved in latest changesets https://github.com/apache/ozone/pull/4244/commits/fb7b7aed61b9a9fb99260670503a5b8678f849ec#diff-55b3048706799272e9f4c51bd618ea71687c5cf6570c4185dc657fbbdf5343fbR183
   
   https://github.com/apache/ozone/pull/4244/commits/fb7b7aed61b9a9fb99260670503a5b8678f849ec#diff-55b3048706799272e9f4c51bd618ea71687c5cf6570c4185dc657fbbdf5343fbR190



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());

Review Comment:
   This looks redundant. It is already loaded in constructor:
   
   https://github.com/apache/ozone/blob/d10ba4b327653fcdcca4b25b50b85b4a0feccf5b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java#L63
   
   ```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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");

Review Comment:
   Should we use `null` here instead? Just like in this UT:
   
   https://github.com/apache/ozone/blob/f30c0621eafc1c7f62e7df0f2da3d11ece842e33/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java#L114



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1701,6 +1704,18 @@ message DeleteSnapshotRequest {
   optional uint64 deletionTime = 4;
 }
 
+message SnapshotMoveDeletedKeysRequest {
+  optional SnapshotInfo fromSnapshot = 1;
+  optional SnapshotInfo nextSnapshot = 2;

Review Comment:
   I think nextSnapshot is no longer used?



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_TIMEOUT;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME_ONE = "bucket1";
+  private static final String BUCKET_NAME_TWO = "bucket2";
+
+
+  @BeforeAll
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @BeforeEach
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (om != null) {
+      om.stop();
+    }
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    createSnapshotDataForBucket1();
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0);
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME_ONE,
+            getSnapshotPrefix("bucket1snap3"));
+
+    // Check bucket1key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/bucket1key1");
+    Assertions.assertNotNull(omKeyInfo);
+
+    // Check bucket1key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/bucket1key2");
+
+    Assertions.assertNotNull(omKeyInfo1);
+
+  }
+
+  @Test
+  public void testMultipleSnapshotKeyReclaim() throws Exception {
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    int snapshotCount = createSnapshotDataForBucket1();
+
+    OmKeyArgs bucket2key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        BucketLayout.DEFAULT, "bucket2key1");
+
+    OmKeyArgs bucket2key2 = createKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        "bucket2key2");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap1",
+        ++snapshotCount);
+
+    // Both key 1 and key 2 can be reclaimed when Snapshot 1 is deleted.
+    writeClient.deleteKey(bucket2key1);
+    writeClient.deleteKey(bucket2key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap2",
+        ++snapshotCount);
+
+    String snapshotKey2 = "/vol1/bucket2/bucket2snap1";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    Assertions.assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0L);
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    // Check bucket2key1 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key1");
+
+    // Check bucket2key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo2 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key2");
+
+    Assertions.assertNotNull(omKeyInfo1);

Review Comment:
   Let me take a look at it, might be missing something in the test.



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java:
##########
@@ -234,6 +234,16 @@ public final class OzoneConfigKeys {
       OZONE_SNAPSHOT_SST_FILTERING_SERVICE_TIMEOUT_DEFAULT = "300s";
       // 300s for default
 
+  public static final String OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL =
+      "ozone.snapshot.deletion.service.interval";
+  public static final String
+      OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL_DEFAULT = "30s";
+
+  public static final String OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT =
+      "ozone.snapshot.deletion.service.timeout";
+  public static final String
+      OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT_DEFAULT = "300s";

Review Comment:
   `deletion` -> `deleting` for consistency?



##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3170,6 +3178,25 @@
     </description>
   </property>
 
+  <property>
+    <name>ozone.snapshot.deletion.service.timeout</name>
+    <value>300s</value>
+    <tag>OZONE, PERFORMANCE, OM</tag>
+    <description>
+      Timeout value for SnapshotDeletingService.
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.snapshot.deletion.service.interval</name>

Review Comment:
   rename `deletion` -> `deleting` for consistency?



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -246,7 +257,7 @@ private boolean deleteSnapshotPath(String snapshotPath,
     return status;
   }
 
-  private void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)
+  public void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)

Review Comment:
   Add a javadoc for this now that it is marked `public`.
   
   State that it clears the existing chain and reloads when called twice.



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -246,7 +257,7 @@ private boolean deleteSnapshotPath(String snapshotPath,
     return status;
   }
 
-  private void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)
+  public void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)

Review Comment:
   I think it should be okay to clear it. As it is reconstructed again.



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java:
##########
@@ -837,6 +838,7 @@ protected OmKeyInfo wrapUncommittedBlocksAsPseudoKey(
     LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
         uncommitted, omKeyInfo.getKeyName());
     OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
+    pseudoKeyInfo.setObjectID(OBJECT_ID_DEFAULT);

Review Comment:
   Let's put a comment above this line that we are using this as a special marker to indicate that this key's blocks can be unconditionally reclaimed in SnapshotDeletingTask



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/key/OMKeyRequest.java:
##########
@@ -837,6 +838,7 @@ protected OmKeyInfo wrapUncommittedBlocksAsPseudoKey(
     LOG.info("Detect allocated but uncommitted blocks {} in key {}.",
         uncommitted, omKeyInfo.getKeyName());
     OmKeyInfo pseudoKeyInfo = omKeyInfo.copyObject();
+    pseudoKeyInfo.setObjectID(OBJECT_ID_DEFAULT);

Review Comment:
   Let's put a comment above this line that we are using this as a special marker to indicate that this key's blocks can be unconditionally reclaimed in `SnapshotDeletingService`.



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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

   Thanks @aswinshakil for the main logic implementation. Thanks @GeorgeJahad @sumitagrawl @neils-dev @DaveTeng0 @hemantk-12 @prashantpogde for reviewing this.


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

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

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


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


[GitHub] [ozone] DaveTeng0 commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+
+    //Check key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/key2");
+
+    assertNotNull(omKeyInfo1);
+
+  }
+
+  private OmKeyArgs createVolumeBucketKey(String volumeName, String bucketName,
+      BucketLayout bucketLayout, String keyName) throws IOException {
+    // cheat here, just create a volume and bucket entry so that we can
+    // create the keys, we put the same data for key and value since the
+    // system does not decode the object
+    OMRequestTestUtils.addVolumeToOM(omMetadataManager,
+        OmVolumeArgs.newBuilder()
+            .setOwnerName("owner")
+            .setAdminName("admin")
+            .setVolume(volumeName)
+            .build());
+
+    OMRequestTestUtils.addBucketToOM(omMetadataManager,
+        OmBucketInfo.newBuilder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return createKey(volumeName, bucketName, keyName);
+  }
+
+  private OmKeyArgs createKey(String volumeName, String bucketName,
+       String keyName) throws IOException {
+    OmKeyArgs keyArg =
+        new OmKeyArgs.Builder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setAcls(Collections.emptyList())
+            .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+            .setLocationInfoList(new ArrayList<>())
+            .build();
+
+    // Open and write the key without commit it.
+    OpenKeySession session = writeClient.openKey(keyArg);
+    writeClient.commitKey(keyArg, session.getId());

Review Comment:
   oh, here it calls commitKey, but I saw the comments above says without commit 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] aswinshakil commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();

Review Comment:
   Good catch. It should. 



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,

Review Comment:
   ```suggestion
       super(SnapshotDeletingService.class.getSimpleName(),
           interval, TimeUnit.MILLISECONDS,
   ```



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java:
##########
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Handles OMSnapshotMoveDeletedKeys Request.
+ */
+public class OMSnapshotMoveDeletedKeysRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotMoveDeletedKeysRequest.class);
+
+  public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+    OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager();
+
+    SnapshotMoveDeletedKeysRequest moveDeletedKeysRequest =
+        getOmRequest().getSnapshotMoveDeletedKeysRequest();
+
+    // If there is no Non-Deleted Snapshot move the
+    // keys to Active Object Store.
+    SnapshotInfo nextSnapshot = null;
+    if (moveDeletedKeysRequest.hasNextSnapshot()) {
+      nextSnapshot = SnapshotInfo
+          .getFromProtobuf(moveDeletedKeysRequest.getNextSnapshot());
+    }
+

Review Comment:
   Nice catch. Will update the 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] aswinshakil commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+
+    //Check key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/key2");
+
+    assertNotNull(omKeyInfo1);
+
+  }
+
+  private OmKeyArgs createVolumeBucketKey(String volumeName, String bucketName,
+      BucketLayout bucketLayout, String keyName) throws IOException {
+    // cheat here, just create a volume and bucket entry so that we can
+    // create the keys, we put the same data for key and value since the
+    // system does not decode the object
+    OMRequestTestUtils.addVolumeToOM(omMetadataManager,
+        OmVolumeArgs.newBuilder()
+            .setOwnerName("owner")
+            .setAdminName("admin")
+            .setVolume(volumeName)
+            .build());
+
+    OMRequestTestUtils.addBucketToOM(omMetadataManager,
+        OmBucketInfo.newBuilder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return createKey(volumeName, bucketName, keyName);
+  }
+
+  private OmKeyArgs createKey(String volumeName, String bucketName,
+       String keyName) throws IOException {
+    OmKeyArgs keyArg =
+        new OmKeyArgs.Builder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setAcls(Collections.emptyList())
+            .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+            .setLocationInfoList(new ArrayList<>())
+            .build();
+
+    // Open and write the key without commit it.
+    OpenKeySession session = writeClient.openKey(keyArg);
+    writeClient.commitKey(keyArg, session.getId());
+
+    return keyArg;
+  }
+
+  private void createSnapshot(String volName, String bucketName,
+       String snapName, int count) throws Exception {
+    writeClient.createSnapshot(volName, bucketName, snapName);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return omMetadataManager.getSnapshotInfoTable()
+            .getEstimatedKeyCount() >= count;

Review Comment:
   From the docs, it looks like it estimates the key count for the entire column family. `getEstimatedKeyCount()` is sometimes inaccurate, It is only used for testing purposes. More here about it, https://github.com/facebook/rocksdb/wiki/RocksDB-FAQ#failure-recovery



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+
+    //Check key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/key2");
+
+    assertNotNull(omKeyInfo1);
+
+  }
+
+  private OmKeyArgs createVolumeBucketKey(String volumeName, String bucketName,
+      BucketLayout bucketLayout, String keyName) throws IOException {
+    // cheat here, just create a volume and bucket entry so that we can
+    // create the keys, we put the same data for key and value since the
+    // system does not decode the object
+    OMRequestTestUtils.addVolumeToOM(omMetadataManager,
+        OmVolumeArgs.newBuilder()
+            .setOwnerName("owner")
+            .setAdminName("admin")
+            .setVolume(volumeName)
+            .build());
+
+    OMRequestTestUtils.addBucketToOM(omMetadataManager,
+        OmBucketInfo.newBuilder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return createKey(volumeName, bucketName, keyName);
+  }
+
+  private OmKeyArgs createKey(String volumeName, String bucketName,
+       String keyName) throws IOException {
+    OmKeyArgs keyArg =
+        new OmKeyArgs.Builder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setAcls(Collections.emptyList())
+            .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+            .setLocationInfoList(new ArrayList<>())
+            .build();
+
+    // Open and write the key without commit it.
+    OpenKeySession session = writeClient.openKey(keyArg);
+    writeClient.commitKey(keyArg, session.getId());

Review Comment:
   It was a copy-paste mistake. Fixed 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] sumitagrawl commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()

Review Comment:
   @smengcl This is possible when getting bucketInfo from active db with volume/bucket name from snapshotDB



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java:
##########
@@ -262,6 +268,24 @@ public void start(OzoneConfiguration configuration) {
         snapshotSstFilteringService.start();
       }
     }
+
+    if (snapshotDeletingService == null) {
+      long snapshotServiceInterval = configuration.getTimeDuration(
+          OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+          OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL_DEFAULT,
+          TimeUnit.MILLISECONDS);
+      long snapshotServiceTimeout = configuration.getTimeDuration(
+          OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+          OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT_DEFAULT,
+          TimeUnit.MILLISECONDS);
+      try {
+        snapshotDeletingService = new SnapshotDeletingService(
+            snapshotServiceInterval, snapshotServiceTimeout, ozoneManager);
+        snapshotDeletingService.start();

Review Comment:
   Yes it only runs on leader OM, the check is done here `shouldRun()`



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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

   Thanks @aswinshakil for the updates. Apart from unresolved UT comments, the latest changes looks good 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] aswinshakil commented on pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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

   Thank you @smengcl for the review. Added the Unit test. 


-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -246,7 +257,7 @@ private boolean deleteSnapshotPath(String snapshotPath,
     return status;
   }
 
-  private void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)
+  public void loadFromSnapshotInfoTable(OMMetadataManager metadataManager)

Review Comment:
   Do we need to clear the new map in this method as well?
   
   ```
   snapshotPathToTableKey.clear();
   ```
   
   and what about `latestPathSnapshotID`



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super(SnapshotDeletingService.class.getSimpleName(), interval,
+        TimeUnit.MILLISECONDS, SNAPSHOT_DELETING_CORE_POOL_SIZE,
+        serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.isEmpty()) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+          if (bucketInfo == null) {
+            throw new OMException("Bucket " + snapInfo.getBucketName() +
+                " is not found", BUCKET_NOT_FOUND);
+          }
+
+          //TODO: [SNAPSHOT] Add lock to deletedTable and Active DB.
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.

Review Comment:
   I agree, It can be confusing here. But the intention was to have this comment here is to explain why `previousSnapshot` can be `null`



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

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

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


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


[GitHub] [ozone] smengcl commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super(SnapshotDeletingService.class.getSimpleName(), interval,
+        TimeUnit.MILLISECONDS, SNAPSHOT_DELETING_CORE_POOL_SIZE,
+        serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.isEmpty()) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+          if (bucketInfo == null) {
+            throw new OMException("Bucket " + snapInfo.getBucketName() +
+                " is not found", BUCKET_NOT_FOUND);
+          }
+
+          //TODO: [SNAPSHOT] Add lock to deletedTable and Active DB.
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<SnapshotMoveKeyInfos> toActiveDBList = new ArrayList<>();
+          List<SnapshotMoveKeyInfos> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            String snapshotBucketKey = dbBucketKey + OzoneConsts.OM_KEY_PREFIX;
+            iterator.seek(snapshotBucketKey);
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              String deletedKey = deletedKeyValue.getKey();
+
+              // Exit if it is out of the bucket scope.
+              if (!deletedKey.contains(snapshotBucketKey)) {
+                break;
+              }
+
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              SnapshotMoveKeyInfos.Builder toActiveDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+              SnapshotMoveKeyInfos.Builder toNextDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, toActiveDBList,
+                toNextDBList);
+            snapshotLimit--;
+            successRunCount.incrementAndGet();
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(SnapshotMoveKeyInfos.Builder toActiveDb,
+        SnapshotMoveKeyInfos.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        List<SnapshotMoveKeyInfos> toActiveDBList,
+        List<SnapshotMoveKeyInfos> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: [SNAPSHOT] Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      if (chainManager.hasPreviousPathSnapshot(snapInfo.getSnapshotPath(),
+          snapInfo.getSnapshotID())) {
+        String previousPathSnapshot = chainManager.previousPathSnapshot(
+            snapInfo.getSnapshotPath(), snapInfo.getSnapshotID());
+        String tableKey = chainManager.getTableKey(previousPathSnapshot);
+        return omSnapshotManager.getSnapshotInfo(tableKey);
+      }
+      return null;
+    }
+
+    private void submitRequest(OMRequest omRequest) {
+      try {
+        if (isRatisEnabled()) {
+          OzoneManagerRatisServer server = ozoneManager.getOmRatisServer();
+
+          RaftClientRequest raftClientRequest = RaftClientRequest.newBuilder()
+              .setClientId(clientId)
+              .setServerId(server.getRaftPeerId())
+              .setGroupId(server.getRaftGroupId())
+              .setCallId(runCount.get())
+              .setMessage(Message.valueOf(
+                  OMRatisHelper.convertRequestToByteString(omRequest)))
+              .setType(RaftClientRequest.writeRequestType())
+              .build();
+
+          server.submitRequest(omRequest, raftClientRequest);
+        } else {
+          ozoneManager.getOmServerProtocol().submitRequest(null, omRequest);
+        }
+      } catch (ServiceException e) {
+        LOG.error("Snapshot Deleting request failed. " +
+            "Will retry at next run.", e);
+      }
+    }
+
+    private boolean isRatisEnabled() {
+      return ozoneManager.isRatisEnabled();
+    }
+
+  }
+
+  @Override
+  public BackgroundTaskQueue getTasks() {
+    BackgroundTaskQueue queue = new BackgroundTaskQueue();
+    queue.add(new SnapshotDeletingTask());
+    return queue;
+  }
+
+  private boolean shouldRun() {
+    return !suspended.get() && ozoneManager.isLeaderReady();
+  }
+
+  /**
+   * Suspend the service.
+   */
+  @VisibleForTesting
+  void suspend() {
+    suspended.set(true);
+  }
+
+  /**
+   * Resume the service if suspended.
+   */
+  @VisibleForTesting
+  void resume() {
+    suspended.set(false);
+  }
+
+  public long getRunCount() {
+    return runCount.get();
+  }
+
+  public long getSuccessfulRunCount() {
+    return successRunCount.get();
+  }
+
+  @VisibleForTesting
+  public void setSuccessRunCount(long num) {
+    successRunCount.getAndSet(num);
+  }
+}

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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/WithObjectID.java:
##########
@@ -61,10 +61,6 @@ public long getUpdateID() {
    * @param obId - long
    */
   public void setObjectID(long obId) {
-    if (this.objectID != 0) {

Review Comment:
   Or adjust the condition to make `obId` equals `OBJECT_ID_DEFAULT` the only exception:
   
   ```
   if (this.objectID != 0L && obId != OBJECT_ID_DEFAULT) {
   ```



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/WithObjectID.java:
##########
@@ -61,10 +61,6 @@ public long getUpdateID() {
    * @param obId - long
    */
   public void setObjectID(long obId) {
-    if (this.objectID != 0) {

Review Comment:
   Or adjust the condition to make `obId` equals `OBJECT_ID_DEFAULT` the only exception:
   
   ```
   if (this.objectID != 0L && obId != OBJECT_ID_DEFAULT) {
   ```
   
   To avoid other methods from potentially abusing this before we do the proper OmKeyInfo field addition (so we can flip it back).



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+
+    //Check key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/key2");
+
+    assertNotNull(omKeyInfo1);
+
+  }

Review Comment:
   New UT request: Take snapshot on two buckets separately



##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+
+    //Check key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/key2");
+
+    assertNotNull(omKeyInfo1);
+
+  }

Review Comment:
   New UT request: Take snapshot on two different buckets



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()

Review Comment:
   @sumitagrawl Theoretically `bucketInfo` won't be `null` because we prevent buckets deletion when they still have any snapshots. But yes we should handle the case (throw proper exception rather than NPE).



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

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

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


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


[GitHub] [ozone] neils-dev commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

Posted by "neils-dev (via GitHub)" <gi...@apache.org>.
neils-dev commented on code in PR #4244:
URL: https://github.com/apache/ozone/pull/4244#discussion_r1107915544


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setPathPreviousSnapshotID(latestPathSnapshot);
+      }
+
+      if (latestGlobalSnapshot == null || latestGlobalSnapshot.isEmpty()) {
+        snapshotInfo.setGlobalPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setGlobalPreviousSnapshotID(latestGlobalSnapshot);
+      }
+
+      snapshotChainManager.addSnapshot(snapshotInfo);

Review Comment:
   @smengcl , 
   
   > we will have to reconstruct the snapshot chain during OM startup (first implementation could reconstruct it from snapshotInfoTable.
   
   On OM startup to reconstruct the chain from persistent storage (`snapshotInfoTable`), is done internally by the `SnapshotChainManager `on instantiation.  When the metadatamanager creates its `SnapshotChainManager`, the instance initializes the chain in memory loading the snapshots found in the snapshotInfoTable.  This is done with `SnapshotChainManager.loadFromSnapshotInfoTable`.  This method is private and used in the constructor.
   
   Not sure if the new jira is needed as when the OM startups up the chain should be initialized with the contents of the persistent store `SnapshotInfoTable`.
   
   https://github.com/apache/ozone/blob/874108337421a1204b41ceaf1f0a85ab5de77f50/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java#L249
   



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3156,6 +3156,14 @@
       sst filtering service per time interval.
     </description>
   </property>
+  <property>
+    <name>ozone.snapshot.deleting.limit.per.task</name>
+    <value>10</value>
+    <tag>OZONE, PERFORMANCE, OM</tag>
+    <description>The number of maximum snapshots to be reclaimed by the

Review Comment:
   ```suggestion
       <description>The maximum number of snapshots that would be reclaimed by
   ```



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME = "bucket1";
+
+
+  @BeforeClass
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @Before
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @After
+  public void cleanup() throws Exception {
+    om.stop();
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    int snapshotCount = 0;
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    snapshotDeletingService.suspend();
+
+    OmKeyArgs key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME,
+        BucketLayout.DEFAULT, "key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap1", ++snapshotCount);
+
+    OmKeyArgs key2 = createKey(VOLUME_NAME, BUCKET_NAME, "key2");
+
+    // Key 1 cannot be deleted as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(key1);
+    //Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap2", ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME, "key4");
+    OmKeyArgs key5 = createKey(VOLUME_NAME, BUCKET_NAME, "key5");
+    writeClient.deleteKey(key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME, "snap3", ++snapshotCount);
+
+
+    String snapshotKey2 = "/vol1/bucket1/snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME, getSnapshotPrefix("snap3"));
+
+    //Check key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/key1");
+    assertNotNull(omKeyInfo);
+
+    //Check key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/key2");
+
+    assertNotNull(omKeyInfo1);
+
+  }

Review Comment:
   Relevant (integration) test for reference: https://github.com/apache/ozone/pull/4273/files#diff-71c1f01e984fc77a5d653be56ec27d6347b2f8104c717a936b6a53a25c5116e3R602-R627



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -91,8 +93,8 @@ private void addSnapshotGlobal(String snapshotID,
   };
 
   private void addSnapshotPath(String snapshotPath,
-                               String snapshotID,
-                               String prevPathID) throws IOException {
+      String snapshotID, String prevPathID, String snapTableKey)

Review Comment:
   It'd be great if we can add a javadoc for this method.



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -49,6 +49,7 @@ public class SnapshotChainManager {
       snapshotChainPath;
   private Map<String, String> latestPathSnapshotID;
   private String latestGlobalSnapshotID;
+  private Map<String, String> snapshotPathToKey;

Review Comment:
   Key is meant to be the Snapshot Table Key. Thanks for pointing it out. Updating 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] aswinshakil commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()

Review Comment:
   The buckets cannot be deleted as long as there are snapshots associated with them. So even in active db it should have the bucketInfo. But as @smengcl mentioned, we can handle NPE cases better. 



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_TIMEOUT;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME_ONE = "bucket1";
+  private static final String BUCKET_NAME_TWO = "bucket2";
+
+
+  @BeforeAll
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @BeforeEach
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (om != null) {
+      om.stop();
+    }
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    createSnapshotDataForBucket1();
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0);
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME_ONE,
+            getSnapshotPrefix("bucket1snap3"));
+
+    // Check bucket1key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/bucket1key1");
+    Assertions.assertNotNull(omKeyInfo);
+
+    // Check bucket1key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/bucket1key2");
+
+    Assertions.assertNotNull(omKeyInfo1);
+
+  }
+
+  @Test
+  public void testMultipleSnapshotKeyReclaim() throws Exception {
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    int snapshotCount = createSnapshotDataForBucket1();
+
+    OmKeyArgs bucket2key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        BucketLayout.DEFAULT, "bucket2key1");
+
+    OmKeyArgs bucket2key2 = createKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        "bucket2key2");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap1",
+        ++snapshotCount);
+
+    // Both key 1 and key 2 can be reclaimed when Snapshot 1 is deleted.
+    writeClient.deleteKey(bucket2key1);
+    writeClient.deleteKey(bucket2key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap2",
+        ++snapshotCount);
+
+    String snapshotKey2 = "/vol1/bucket2/bucket2snap1";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    Assertions.assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0L);
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    // Check bucket2key1 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key1");
+
+    // Check bucket2key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo2 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key2");
+
+    Assertions.assertNotNull(omKeyInfo1);
+    Assertions.assertNotNull(omKeyInfo2);
+  }
+
+  private OmKeyArgs createVolumeBucketKey(String volumeName, String bucketName,
+      BucketLayout bucketLayout, String keyName) throws IOException {
+    // cheat here, just create a volume and bucket entry so that we can
+    // create the keys, we put the same data for key and value since the
+    // system does not decode the object
+    OMRequestTestUtils.addVolumeToOM(omMetadataManager,
+        OmVolumeArgs.newBuilder()
+            .setOwnerName("owner")
+            .setAdminName("admin")
+            .setVolume(volumeName)
+            .build());
+
+    OMRequestTestUtils.addBucketToOM(omMetadataManager,
+        OmBucketInfo.newBuilder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return createKey(volumeName, bucketName, keyName);
+  }
+
+
+  private int createSnapshotDataForBucket1() throws Exception {
+    int snapshotCount = 0;
+    OmKeyArgs bucket1key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        BucketLayout.DEFAULT, "bucket1key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap1",
+        ++snapshotCount);
+
+    OmKeyArgs bucket1key2 = createKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        "bucket1key2");
+
+    // Key 1 cannot be reclaimed as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(bucket1key1);
+    // Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(bucket1key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap2",
+        ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1key4");
+    OmKeyArgs bucket1key5 = createKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        "bucket1key5");
+    writeClient.deleteKey(bucket1key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap3",
+        ++snapshotCount);
+
+    String snapshotKey2 = "/vol1/bucket1/bucket1snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    Assertions.assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    return snapshotCount;
+  }
+
+  private OmKeyArgs createKey(String volumeName, String bucketName,
+       String keyName) throws IOException {
+    OmKeyArgs keyArg =
+        new OmKeyArgs.Builder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setAcls(Collections.emptyList())
+            .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+            .setLocationInfoList(new ArrayList<>())
+            .build();
+
+    // Open and write the key.
+    OpenKeySession session = writeClient.openKey(keyArg);
+    writeClient.commitKey(keyArg, session.getId());
+
+    return keyArg;
+  }
+
+  private void createSnapshot(String volName, String bucketName,
+       String snapName, int count) throws Exception {
+    writeClient.createSnapshot(volName, bucketName, snapName);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return omMetadataManager.getSnapshotInfoTable()
+            .getEstimatedKeyCount() >= count;

Review Comment:
   Since the test is small it shouldn't be an issue here. From the multiple test runs, I see it is fairly accurate for at least a small number of keys. But yes we can change this. I'll update the PR. Thanks for pointing. 



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot fromSnapshot;
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+  private List<SnapshotMoveKeyInfos> reclaimKeyList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       @Nonnull OmSnapshot omFromSnapshot, OmSnapshot omNextSnapshot,
+       List<SnapshotMoveKeyInfos> nextDBKeysList,
+       List<SnapshotMoveKeyInfos> reclaimKeysList) {
+    super(omResponse);
+    this.fromSnapshot = omFromSnapshot;
+    this.nextSnapshot = omNextSnapshot;
+    this.nextDBKeysList = nextDBKeysList;
+    this.reclaimKeyList = reclaimKeysList;

Review Comment:
   nit
   
   ```suggestion
       this.reclaimKeysList = reclaimKeysList;
   ```
   
   if this causes findbugs, try a random prefix, like `om`



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.hdds.utils.db.DBStore;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot fromSnapshot;
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+  private List<SnapshotMoveKeyInfos> reclaimKeyList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       @Nonnull OmSnapshot omFromSnapshot, OmSnapshot omNextSnapshot,
+       List<SnapshotMoveKeyInfos> nextDBKeysList,
+       List<SnapshotMoveKeyInfos> reclaimKeysList) {
+    super(omResponse);
+    this.fromSnapshot = omFromSnapshot;
+    this.nextSnapshot = omNextSnapshot;
+    this.nextDBKeysList = nextDBKeysList;
+    this.reclaimKeyList = reclaimKeysList;

Review Comment:
   nit
   
   ```suggestion
       this.reclaimKeysList = reclaimKeysList;
   ```
   
   if this raises findbugs warning, try a random prefix, like `om`



-- 
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] GeorgeJahad commented on pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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

   I know we are in a hurry, but snapshot delete is the most complicated and dangerous part of the snapshot system, so I'd like to see more tests for this subsystem.
   
   If you are too busy, we can create a separate PR and having someone on my team write the tests, in particular for the following methods:
   
       private SnapshotInfo getNextActiveSnapshot(SnapshotInfo snapInfo,
       private RepeatedOmKeyInfo createRepeatedOmKeyInfo(List<KeyInfo> keyInfoList)
       private void splitRepeatedOmKeyInfo(SnapshotMoveKeyInfos.Builder toActiveDb,
       private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)
       private boolean checkKeyExistInPreviousTable(
   
   In addition, I'd like a unit test that confirms that we are correctly starting and stopping within the bucket scope.
   


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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,

Review Comment:
   You are correct. I have updated the patch to separate batch operation for snapshots. 



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super(SnapshotDeletingService.class.getSimpleName(), interval,
+        TimeUnit.MILLISECONDS, SNAPSHOT_DELETING_CORE_POOL_SIZE,
+        serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.isEmpty()) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+          if (bucketInfo == null) {
+            throw new OMException("Bucket " + snapInfo.getBucketName() +
+                " is not found", BUCKET_NOT_FOUND);
+          }
+
+          //TODO: [SNAPSHOT] Add lock to deletedTable and Active DB.
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<SnapshotMoveKeyInfos> toActiveDBList = new ArrayList<>();
+          List<SnapshotMoveKeyInfos> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            String snapshotBucketKey = dbBucketKey + OzoneConsts.OM_KEY_PREFIX;
+            iterator.seek(snapshotBucketKey);
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              String deletedKey = deletedKeyValue.getKey();
+
+              // Exit if it is out of the bucket scope.
+              if (!deletedKey.contains(snapshotBucketKey)) {
+                break;
+              }
+
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              SnapshotMoveKeyInfos.Builder toActiveDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+              SnapshotMoveKeyInfos.Builder toNextDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, toActiveDBList,
+                toNextDBList);
+            snapshotLimit--;

Review Comment:
   Do we need to change the snapshotInfo status to SNAPSHOT_RECLAIMED 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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,

Review Comment:
   @swamirishi would be taking care of this in a separate 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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,

Review Comment:
   Is it a problem that we are doing the put with the snapshots metadata manager, but the commit is happening to the omMetadataManage?r: https://github.com/apache/ozone/blob/3de525c60c40441a938f84ffda61ebd2b8bad50b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerDoubleBuffer.java#L346-L347



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

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

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


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


[GitHub] [ozone] aswinshakil commented on pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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

   @GeorgeJahad I agree we need more tests, I'll add them eventually along with my other PR's. `SnapshotDeletingService` is not yet fully completed yet. There are follow-up JIRA [HDDS-7883](https://issues.apache.org/jira/browse/HDDS-7883), The things not addressed here will be updated in the follow-up patch. 


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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-hdds/common/src/main/resources/ozone-default.xml:
##########
@@ -3180,6 +3188,25 @@
     </description>
   </property>
 
+  <property>
+    <name>ozone.snapshot.deletion.service.timeout</name>
+    <value>300s</value>
+    <tag>OZONE, PERFORMANCE, OM</tag>
+    <description>
+      Timeout value for SnapshotDeletingService.
+    </description>
+  </property>
+
+  <property>
+    <name>ozone.snapshot.deletion.service.interval</name>
+    <value>30s</value>

Review Comment:
   It runs at a fixed delay. 
   
   https://github.com/apache/ozone/blob/a66e43e2838f087229cb4b0fac1e1dc969ab80dd/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BackgroundService.java#L90-L92



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+
+          chainManager.loadFromSnapshotInfoTable(
+              ozoneManager.getMetadataManager());
+          SnapshotInfo nextSnapshot = getNextNonDeletedSnapshot(snapInfo);
+
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<KeyValuePair> toActiveDBList = new ArrayList<>();
+          List<KeyValuePair> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            iterator.seekToFirst();
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              KeyValuePair.Builder toActiveDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+              KeyValuePair.Builder toNextDb = KeyValuePair.newBuilder()
+                  .setKey(deletedKeyValue.getKey());
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, nextSnapshot,
+                toActiveDBList, toNextDBList);
+            snapshotLimit--;
+          } catch (IOException ex) {
+            LOG.error("Error while running Snapshot Deleting Service", ex);
+          }
+        }
+
+        successRunCount.incrementAndGet();
+      } catch (IOException e) {
+        LOG.error("Error while running Snapshot Deleting Service", e);
+      }
+
+      return BackgroundTaskResult.EmptyTaskResult.newResult();
+    }
+
+    private void splitRepeatedOmKeyInfo(KeyValuePair.Builder toActiveDb,
+        KeyValuePair.Builder toNextDb, OmKeyInfo keyInfo,
+        Table<String, OmKeyInfo> previousKeyTable) throws IOException {
+      if (checkKeyExistInPreviousTable(previousKeyTable, keyInfo)) {
+        // Move to next non deleted snapshot's deleted table
+        toNextDb.addKeyInfos(keyInfo.getProtobuf(
+            ClientVersion.CURRENT_VERSION));
+      } else {
+        // Move to active DB Deleted Table.
+        toActiveDb.addKeyInfos(keyInfo
+            .getProtobuf(ClientVersion.CURRENT_VERSION));
+      }
+    }
+
+    private void submitSnapshotMoveDeletedKeys(SnapshotInfo snapInfo,
+        SnapshotInfo nextSnapshot, List<KeyValuePair> toActiveDBList,
+        List<KeyValuePair> toNextDBList) {
+
+      SnapshotMoveDeletedKeysRequest.Builder moveDeletedKeysBuilder =
+          SnapshotMoveDeletedKeysRequest.newBuilder()
+              .setFromSnapshot(snapInfo.getProtobuf());
+
+      if (nextSnapshot != null) {
+        moveDeletedKeysBuilder.setNextSnapshot(nextSnapshot.getProtobuf());
+      }
+
+      SnapshotMoveDeletedKeysRequest moveDeletedKeys =
+          moveDeletedKeysBuilder.addAllActiveDBKeys(toActiveDBList)
+          .addAllNextDBKeys(toNextDBList).build();
+
+
+      OMRequest omRequest = OMRequest.newBuilder()
+          .setCmdType(Type.SnapshotMoveDeletedKeys)
+          .setSnapshotMoveDeletedKeysRequest(moveDeletedKeys)
+          .setClientId(clientId.toString())
+          .build();
+
+      submitRequest(omRequest);
+    }
+
+    private boolean checkKeyExistInPreviousTable(
+        Table<String, OmKeyInfo> previousKeyTable, OmKeyInfo deletedKeyInfo)
+        throws IOException {
+
+      if (previousKeyTable == null) {
+        return false;
+      }
+
+      //TODO: Handle Renamed Keys
+      String dbKey = ozoneManager.getMetadataManager()
+          .getOzoneKey(deletedKeyInfo.getVolumeName(),
+              deletedKeyInfo.getBucketName(), deletedKeyInfo.getKeyName());
+
+      OmKeyInfo prevKeyInfo = previousKeyTable.get(dbKey);
+      if (prevKeyInfo != null &&
+          prevKeyInfo.getObjectID() == deletedKeyInfo.getObjectID()) {
+        return true;
+      }
+      return false;
+    }
+
+    private SnapshotInfo getPreviousSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      if (chainManager.hasPreviousPathSnapshot(snapInfo.getSnapshotPath(),
+          snapInfo.getSnapshotID())) {
+        String previousPathSnapshot = chainManager.previousPathSnapshot(
+            snapInfo.getSnapshotPath(), snapInfo.getSnapshotID());
+        String tableKey = chainManager.getTableKey(previousPathSnapshot);
+        return omSnapshotManager.getSnapshotInfo(tableKey);
+      }
+      return null;
+    }
+
+    /**
+     * Get the next non deleted snapshot in the snapshot chain.
+     */
+    private SnapshotInfo getNextNonDeletedSnapshot(SnapshotInfo snapInfo)
+        throws IOException {
+      while (chainManager.hasNextPathSnapshot(snapInfo.getSnapshotPath(), 
+          snapInfo.getSnapshotID())) {

Review Comment:
   Filed HDDS-8059 for this.



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

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

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


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


[GitHub] [ozone] smengcl commented on a diff in pull request #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,378 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyValuePair;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super("SnapshotDeletingService", interval, TimeUnit.MILLISECONDS,
+        SNAPSHOT_DELETING_CORE_POOL_SIZE, serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+        iterator.seekToFirst();
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.getEstimatedKeyCount() == 0) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()

Review Comment:
   @sumitagrawl Theoretically `bucketInfo` won't be `null` because we prevent buckets deletion when they still have any snapshots. But yes we should handle the case (throw proper informative exception rather than NPE).



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setPathPreviousSnapshotID(latestPathSnapshot);
+      }
+
+      if (latestGlobalSnapshot == null || latestGlobalSnapshot.isEmpty()) {
+        snapshotInfo.setGlobalPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setGlobalPreviousSnapshotID(latestGlobalSnapshot);
+      }
+
+      snapshotChainManager.addSnapshot(snapshotInfo);

Review Comment:
   CMIIW but this seems to be the only place where snapshots are added to OM's `SnapshotChainManager`?
   
   Since `SnapshotChainManager` persists its entire state in-memory, we will have to reconstruct the snapshot chain during OM startup (firs t implementation could reconstruct it from scratch. persist it later if too time consuming).



##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setPathPreviousSnapshotID(latestPathSnapshot);
+      }
+
+      if (latestGlobalSnapshot == null || latestGlobalSnapshot.isEmpty()) {
+        snapshotInfo.setGlobalPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setGlobalPreviousSnapshotID(latestGlobalSnapshot);
+      }
+
+      snapshotChainManager.addSnapshot(snapshotInfo);

Review Comment:
   CMIIW but this seems to be the only place where snapshots are added to OM's `SnapshotChainManager`?
   
   Since `SnapshotChainManager` persists its entire state in-memory, we will have to reconstruct the snapshot chain during OM startup (first implementation could reconstruct it from scratch. persist it later if too time consuming).



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");

Review Comment:
   Should we use `null` here instead? like in this UT:
   
   https://github.com/apache/ozone/blob/f30c0621eafc1c7f62e7df0f2da3d11ece842e33/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotChain.java#L114



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotCreateRequest.java:
##########
@@ -146,6 +149,26 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
               .getLatestSequenceNumber();
       snapshotInfo.setDbTxSequenceNumber(dbLatestSequenceNumber);
 
+      // Set previous path and global snapshot
+      String latestPathSnapshot =
+          snapshotChainManager.getLatestPathSnapshot(snapshotPath);
+      String latestGlobalSnapshot =
+          snapshotChainManager.getLatestGlobalSnapshot();
+
+      if (latestPathSnapshot == null || latestPathSnapshot.isEmpty()) {
+        snapshotInfo.setPathPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setPathPreviousSnapshotID(latestPathSnapshot);
+      }
+
+      if (latestGlobalSnapshot == null || latestGlobalSnapshot.isEmpty()) {
+        snapshotInfo.setGlobalPreviousSnapshotID("");
+      } else {
+        snapshotInfo.setGlobalPreviousSnapshotID(latestGlobalSnapshot);
+      }
+
+      snapshotChainManager.addSnapshot(snapshotInfo);

Review Comment:
   Filed HDDS-7970



-- 
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 #4244: HDDS-7740. Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/SnapshotChainManager.java:
##########
@@ -91,8 +93,8 @@ private void addSnapshotGlobal(String snapshotID,
   };
 
   private void addSnapshotPath(String snapshotPath,
-                               String snapshotID,
-                               String prevPathID) throws IOException {
+      String snapshotID, String prevPathID, String snapTableKey)

Review Comment:
   It'd be great if we can add a javadoc to this method and the public method below.



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Handles OMSnapshotMoveDeletedKeys Request.
+ */
+public class OMSnapshotMoveDeletedKeysRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotMoveDeletedKeysRequest.class);
+
+  public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+    OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    SnapshotChainManager snapshotChainManager =
+        ozoneManager.getSnapshotChainManager();
+
+    SnapshotMoveDeletedKeysRequest moveDeletedKeysRequest =
+        getOmRequest().getSnapshotMoveDeletedKeysRequest();
+    SnapshotInfo fromSnapshot = SnapshotInfo.getFromProtobuf(
+        moveDeletedKeysRequest.getFromSnapshot());
+
+    // If there is no Non-Deleted Snapshot move the
+    // keys to Active Object Store.
+    SnapshotInfo nextSnapshot = null;
+    OMClientResponse omClientResponse = null;
+    OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    try {
+      nextSnapshot = getNextActiveSnapshot(fromSnapshot,
+          snapshotChainManager, omSnapshotManager);
+
+      // Get next non-deleted snapshot.
+
+      List<SnapshotMoveKeyInfos> activeDBKeysList =
+          moveDeletedKeysRequest.getActiveDBKeysList();
+      List<SnapshotMoveKeyInfos> nextDBKeysList =
+          moveDeletedKeysRequest.getNextDBKeysList();
+
+      OmSnapshot omNextSnapshot = null;
+
+      if (nextSnapshot != null) {
+        omNextSnapshot = (OmSnapshot) omSnapshotManager
+            .checkForSnapshot(nextSnapshot.getVolumeName(),

Review Comment:
   Yeah it can be done. Thanks for the suggestion, will update the patch. 



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java:
##########
@@ -0,0 +1,300 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.service;
+
+import org.apache.hadoop.hdds.client.StandaloneReplicationConfig;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.server.ServerUtils;
+import org.apache.hadoop.hdds.utils.db.DBConfigFromFile;
+import org.apache.hadoop.ozone.om.KeyManager;
+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.OmKeyArgs;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.request.OMRequestTestUtils;
+import org.apache.hadoop.test.PathUtils;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.util.ExitUtils;
+import org.junit.Rule;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT;
+
+/**
+ * Test Snapshot Deleting Service.
+ */
+public class TestSnapshotDeletingService {
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private OzoneManagerProtocol writeClient;
+  private OzoneManager om;
+
+  private KeyManager keyManager;
+  private OMMetadataManager omMetadataManager;
+  private OzoneConfiguration conf;
+  private OmTestManagers omTestManagers;
+  private static final String VOLUME_NAME = "vol1";
+  private static final String BUCKET_NAME_ONE = "bucket1";
+  private static final String BUCKET_NAME_TWO = "bucket2";
+
+
+  @BeforeAll
+  public static void setup() {
+    ExitUtils.disableSystemExit();
+  }
+
+  @BeforeEach
+  public void createConfAndInitValues() throws Exception {
+    conf = new OzoneConfiguration();
+    File testDir = PathUtils.getTestDir(TestSnapshotDeletingService.class);
+    System.setProperty(DBConfigFromFile.CONFIG_DIR, "/");
+    ServerUtils.setOzoneMetaDirPath(conf, testDir.getPath());
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_INTERVAL,
+        1000, TimeUnit.MILLISECONDS);
+    conf.setTimeDuration(OZONE_SNAPSHOT_DELETION_SERVICE_TIMEOUT,
+        100000, TimeUnit.MILLISECONDS);
+    conf.setQuietMode(false);
+    omTestManagers = new OmTestManagers(conf);
+    keyManager = omTestManagers.getKeyManager();
+    omMetadataManager = omTestManagers.getMetadataManager();
+    writeClient = omTestManagers.getWriteClient();
+    om = omTestManagers.getOzoneManager();
+  }
+
+  @AfterEach
+  public void cleanup() throws Exception {
+    if (om != null) {
+      om.stop();
+    }
+  }
+
+  @Test
+  public void testSnapshotKeySpaceReclaim() throws Exception {
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    createSnapshotDataForBucket1();
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0);
+
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    OmSnapshot nextSnapshot = (OmSnapshot) om.getOmSnapshotManager()
+        .checkForSnapshot(VOLUME_NAME, BUCKET_NAME_ONE,
+            getSnapshotPrefix("bucket1snap3"));
+
+    // Check bucket1key1 added to next non deleted snapshot db.
+    RepeatedOmKeyInfo omKeyInfo =
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().get("/vol1/bucket1/bucket1key1");
+    Assertions.assertNotNull(omKeyInfo);
+
+    // Check bucket1key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket1/bucket1key2");
+
+    Assertions.assertNotNull(omKeyInfo1);
+
+  }
+
+  @Test
+  public void testMultipleSnapshotKeyReclaim() throws Exception {
+
+    SnapshotDeletingService snapshotDeletingService = (SnapshotDeletingService)
+        keyManager.getSnapshotDeletingService();
+
+    // Suspending SnapshotDeletingService
+    snapshotDeletingService.suspend();
+    int snapshotCount = createSnapshotDataForBucket1();
+
+    OmKeyArgs bucket2key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        BucketLayout.DEFAULT, "bucket2key1");
+
+    OmKeyArgs bucket2key2 = createKey(VOLUME_NAME, BUCKET_NAME_TWO,
+        "bucket2key2");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap1",
+        ++snapshotCount);
+
+    // Both key 1 and key 2 can be reclaimed when Snapshot 1 is deleted.
+    writeClient.deleteKey(bucket2key1);
+    writeClient.deleteKey(bucket2key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_TWO, "bucket2snap2",
+        ++snapshotCount);
+
+    String snapshotKey2 = "/vol1/bucket2/bucket2snap1";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    Assertions.assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+
+    snapshotDeletingService.resume();
+
+    snapshotDeletingService.setSuccessRunCount(0L);
+    GenericTestUtils.waitFor(() ->
+            snapshotDeletingService.getSuccessfulRunCount() >= 1,
+        1000, 10000);
+
+    // Check bucket2key1 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo1 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key1");
+
+    // Check bucket2key2 added active db as it can be reclaimed.
+    RepeatedOmKeyInfo omKeyInfo2 = omMetadataManager
+        .getDeletedTable().get("/vol1/bucket2/bucket2key2");
+
+    Assertions.assertNotNull(omKeyInfo1);
+    Assertions.assertNotNull(omKeyInfo2);
+  }
+
+  private OmKeyArgs createVolumeBucketKey(String volumeName, String bucketName,
+      BucketLayout bucketLayout, String keyName) throws IOException {
+    // cheat here, just create a volume and bucket entry so that we can
+    // create the keys, we put the same data for key and value since the
+    // system does not decode the object
+    OMRequestTestUtils.addVolumeToOM(omMetadataManager,
+        OmVolumeArgs.newBuilder()
+            .setOwnerName("owner")
+            .setAdminName("admin")
+            .setVolume(volumeName)
+            .build());
+
+    OMRequestTestUtils.addBucketToOM(omMetadataManager,
+        OmBucketInfo.newBuilder().setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return createKey(volumeName, bucketName, keyName);
+  }
+
+
+  private int createSnapshotDataForBucket1() throws Exception {
+    int snapshotCount = 0;
+    OmKeyArgs bucket1key1 = createVolumeBucketKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        BucketLayout.DEFAULT, "bucket1key1");
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap1",
+        ++snapshotCount);
+
+    OmKeyArgs bucket1key2 = createKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        "bucket1key2");
+
+    // Key 1 cannot be reclaimed as it is still referenced by Snapshot 1.
+    writeClient.deleteKey(bucket1key1);
+    // Key 2 is deleted here, which means we can reclaim
+    // it when snapshot 2 is deleted.
+    writeClient.deleteKey(bucket1key2);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap2",
+        ++snapshotCount);
+    createKey(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1key4");
+    OmKeyArgs bucket1key5 = createKey(VOLUME_NAME, BUCKET_NAME_ONE,
+        "bucket1key5");
+    writeClient.deleteKey(bucket1key5);
+
+    createSnapshot(VOLUME_NAME, BUCKET_NAME_ONE, "bucket1snap3",
+        ++snapshotCount);
+
+    String snapshotKey2 = "/vol1/bucket1/bucket1snap2";
+    SnapshotInfo snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+
+    snapshotInfo
+        .setSnapshotStatus(SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    om.getMetadataManager()
+        .getSnapshotInfoTable().put(snapshotKey2, snapshotInfo);
+    snapshotInfo = om.getMetadataManager()
+        .getSnapshotInfoTable().get(snapshotKey2);
+    Assertions.assertEquals(snapshotInfo.getSnapshotStatus(),
+        SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED);
+    return snapshotCount;
+  }
+
+  private OmKeyArgs createKey(String volumeName, String bucketName,
+       String keyName) throws IOException {
+    OmKeyArgs keyArg =
+        new OmKeyArgs.Builder()
+            .setVolumeName(volumeName)
+            .setBucketName(bucketName)
+            .setKeyName(keyName)
+            .setAcls(Collections.emptyList())
+            .setReplicationConfig(StandaloneReplicationConfig.getInstance(
+                HddsProtos.ReplicationFactor.ONE))
+            .setLocationInfoList(new ArrayList<>())
+            .build();
+
+    // Open and write the key.
+    OpenKeySession session = writeClient.openKey(keyArg);
+    writeClient.commitKey(keyArg, session.getId());
+
+    return keyArg;
+  }
+
+  private void createSnapshot(String volName, String bucketName,
+       String snapName, int count) throws Exception {
+    writeClient.createSnapshot(volName, bucketName, snapName);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return omMetadataManager.getSnapshotInfoTable()
+            .getEstimatedKeyCount() >= count;
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+      return false;
+    }, 1000, 10000);
+  }
+}

Review Comment:
   nit: new line eof
   
   ```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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/snapshot/OMSnapshotMoveDeletedKeysResponse.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.response.snapshot;
+
+import org.apache.hadoop.hdds.utils.db.BatchOperation;
+import org.apache.hadoop.ozone.om.OMMetadataManager;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.response.CleanupTableInfo;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+
+import javax.annotation.Nonnull;
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmMetadataManagerImpl.SNAPSHOT_INFO_TABLE;
+
+/**
+ * Response for OMSnapshotMoveDeletedKeysRequest.
+ */
+@CleanupTableInfo(cleanupTables = {SNAPSHOT_INFO_TABLE})
+public class OMSnapshotMoveDeletedKeysResponse extends OMClientResponse {
+
+  private OmSnapshot nextSnapshot;
+  private List<SnapshotMoveKeyInfos> activeDBKeysList;
+  private List<SnapshotMoveKeyInfos> nextDBKeysList;
+
+  public OMSnapshotMoveDeletedKeysResponse(OMResponse omResponse,
+       OmSnapshot omNextSnapshot, List<SnapshotMoveKeyInfos> activeDBKeysList,
+       List<SnapshotMoveKeyInfos> nextDBKeysList) {
+    super(omResponse);
+    this.nextSnapshot = omNextSnapshot;
+    this.activeDBKeysList = activeDBKeysList;
+    this.nextDBKeysList = nextDBKeysList;
+  }
+
+  /**
+   * For when the request is not successful.
+   * For a successful request, the other constructor should be used.
+   */
+  public OMSnapshotMoveDeletedKeysResponse(@Nonnull OMResponse omResponse) {
+    super(omResponse);
+    checkStatusNotOK();
+  }
+
+  @Override
+  protected void addToDBBatch(OMMetadataManager omMetadataManager,
+      BatchOperation batchOperation) throws IOException {
+
+    for (SnapshotMoveKeyInfos activeDBKey : activeDBKeysList) {
+      RepeatedOmKeyInfo activeDBOmKeyInfo =
+          createRepeatedOmKeyInfo(activeDBKey.getKeyInfosList());
+
+      if (activeDBOmKeyInfo == null) {
+        continue;
+      }
+
+      omMetadataManager.getDeletedTable().putWithBatch(
+          batchOperation, activeDBKey.getKey(), activeDBOmKeyInfo);
+    }
+
+    for (SnapshotMoveKeyInfos nextDBKey : nextDBKeysList) {
+      RepeatedOmKeyInfo nextDBOmKeyInfo =
+          createRepeatedOmKeyInfo(nextDBKey.getKeyInfosList());
+
+      if (nextDBOmKeyInfo == null) {
+        continue;
+      }
+
+      if (nextSnapshot != null) {
+        nextSnapshot.getMetadataManager()
+            .getDeletedTable().putWithBatch(batchOperation,

Review Comment:
   Now that we are writing to snapshots, compactions will start happening on them.
   
   Won't that cause problems for the compaction dag algorithm used by snapdiff? 
   
   Is it possible to configure rocksdb, so that the original sst files for this snapshot don't get compacted into the new files created by these writes?  Otherwise, do we need to modify the snapdiff approach to deal with these new compactions?
   



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

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

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


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


[GitHub] [ozone] prashantpogde commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto:
##########
@@ -1701,6 +1704,18 @@ message DeleteSnapshotRequest {
   optional uint64 deletionTime = 4;
 }
 
+message SnapshotMoveDeletedKeysRequest {
+  optional SnapshotInfo fromSnapshot = 1;
+  optional SnapshotInfo nextSnapshot = 2;
+  repeated SnapshotMoveKeyInfos activeDBKeys = 3;
+  repeated SnapshotMoveKeyInfos nextDBKeys = 4;

Review Comment:
   nit:  Naming could be consistent for readability.  you just need "FromSnapshot" DBKeys. Why do you need nextDBKeys ? why do we need activeDBKeys ?



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super(SnapshotDeletingService.class.getSimpleName(), interval,
+        TimeUnit.MILLISECONDS, SNAPSHOT_DELETING_CORE_POOL_SIZE,
+        serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.isEmpty()) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+          if (bucketInfo == null) {
+            throw new OMException("Bucket " + snapInfo.getBucketName() +
+                " is not found", BUCKET_NOT_FOUND);
+          }
+
+          //TODO: [SNAPSHOT] Add lock to deletedTable and Active DB.
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.

Review Comment:
   I find this comment confusing here.  It actually describes the start of checkKeyExistInPreviousTable().  Wouldn't it be better there?
   >          // Handle case when the deleted snapshot is the first snapshot.
   



-- 
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] GeorgeJahad commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Handles OMSnapshotMoveDeletedKeys Request.
+ */
+public class OMSnapshotMoveDeletedKeysRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotMoveDeletedKeysRequest.class);
+
+  public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+    OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    SnapshotChainManager snapshotChainManager =
+        ozoneManager.getSnapshotChainManager();
+
+    SnapshotMoveDeletedKeysRequest moveDeletedKeysRequest =
+        getOmRequest().getSnapshotMoveDeletedKeysRequest();
+    SnapshotInfo fromSnapshot = SnapshotInfo.getFromProtobuf(
+        moveDeletedKeysRequest.getFromSnapshot());
+
+    // If there is no Non-Deleted Snapshot move the
+    // keys to Active Object Store.
+    SnapshotInfo nextSnapshot = null;
+    OMClientResponse omClientResponse = null;
+    OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    try {
+      nextSnapshot = getNextActiveSnapshot(fromSnapshot,
+          snapshotChainManager, omSnapshotManager);
+
+      // Get next non-deleted snapshot.
+
+      List<SnapshotMoveKeyInfos> activeDBKeysList =
+          moveDeletedKeysRequest.getActiveDBKeysList();
+      List<SnapshotMoveKeyInfos> nextDBKeysList =
+          moveDeletedKeysRequest.getNextDBKeysList();
+
+      OmSnapshot omNextSnapshot = null;
+
+      if (nextSnapshot != null) {
+        omNextSnapshot = (OmSnapshot) omSnapshotManager
+            .checkForSnapshot(nextSnapshot.getVolumeName(),

Review Comment:
   My understanding is that validateAndUpdateCache() runs within the applyTransaction() method of the ozoneManager ratis state machine, and therefor needs to run quickly.  Is that not correct?
   



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

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

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


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


[GitHub] [ozone] aswinshakil commented on a diff in pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java:
##########
@@ -0,0 +1,368 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.om.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.protobuf.ServiceException;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.utils.BackgroundService;
+import org.apache.hadoop.hdds.utils.BackgroundTask;
+import org.apache.hadoop.hdds.utils.BackgroundTaskQueue;
+import org.apache.hadoop.hdds.utils.BackgroundTaskResult;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.ozone.ClientVersion;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OMRatisHelper;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import org.apache.ratis.protocol.ClientId;
+import org.apache.ratis.protocol.Message;
+import org.apache.ratis.protocol.RaftClientRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
+
+/**
+ * Background Service to clean-up deleted snapshot and reclaim space.
+ */
+public class SnapshotDeletingService extends BackgroundService {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(SnapshotDeletingService.class);
+
+  // Use only a single thread for Snapshot Deletion. Multiple threads would read
+  // from the same table and can send deletion requests for same snapshot
+  // multiple times.
+  private static final int SNAPSHOT_DELETING_CORE_POOL_SIZE = 1;
+  private final ClientId clientId = ClientId.randomId();
+  private final AtomicLong runCount;
+
+  private final OzoneManager ozoneManager;
+  private final OmSnapshotManager omSnapshotManager;
+  private final SnapshotChainManager chainManager;
+  private final AtomicBoolean suspended;
+  private final OzoneConfiguration conf;
+  private final AtomicLong successRunCount;
+  private final long snapshotDeletionPerTask;
+
+  public SnapshotDeletingService(long interval, long serviceTimeout,
+      OzoneManager ozoneManager) throws IOException {
+    super(SnapshotDeletingService.class.getSimpleName(), interval,
+        TimeUnit.MILLISECONDS, SNAPSHOT_DELETING_CORE_POOL_SIZE,
+        serviceTimeout);
+    this.ozoneManager = ozoneManager;
+    this.omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    this.chainManager = ozoneManager.getSnapshotChainManager();
+    this.runCount = new AtomicLong(0);
+    this.successRunCount = new AtomicLong(0);
+    this.suspended = new AtomicBoolean(false);
+    this.conf = ozoneManager.getConfiguration();
+    this.snapshotDeletionPerTask = conf
+        .getLong(SNAPSHOT_DELETING_LIMIT_PER_TASK,
+        SNAPSHOT_DELETING_LIMIT_PER_TASK_DEFAULT);
+  }
+
+  private class SnapshotDeletingTask implements BackgroundTask {
+
+    @Override
+    public BackgroundTaskResult call() throws Exception {
+      if (!shouldRun()) {
+        return BackgroundTaskResult.EmptyTaskResult.newResult();
+      }
+
+      runCount.incrementAndGet();
+
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          ozoneManager.getMetadataManager().getSnapshotInfoTable();
+      try (TableIterator<String, ? extends Table.KeyValue
+          <String, SnapshotInfo>> iterator = snapshotInfoTable.iterator()) {
+
+        long snapshotLimit = snapshotDeletionPerTask;
+
+        while (iterator.hasNext() && snapshotLimit > 0) {
+          SnapshotInfo snapInfo = iterator.next().getValue();
+          SnapshotInfo.SnapshotStatus snapshotStatus =
+              snapInfo.getSnapshotStatus();
+
+          // Only Iterate in deleted snapshot
+          if (!snapshotStatus.equals(
+              SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED)) {
+            continue;
+          }
+
+          OmSnapshot omSnapshot = (OmSnapshot) omSnapshotManager
+              .checkForSnapshot(snapInfo.getVolumeName(),
+                  snapInfo.getBucketName(),
+                  getSnapshotPrefix(snapInfo.getName()));
+
+          Table<String, RepeatedOmKeyInfo> snapshotDeletedTable =
+              omSnapshot.getMetadataManager().getDeletedTable();
+
+          if (snapshotDeletedTable.isEmpty()) {
+            continue;
+          }
+
+          // Get bucketInfo for the snapshot bucket to get bucket layout.
+          String dbBucketKey = ozoneManager.getMetadataManager().getBucketKey(
+              snapInfo.getVolumeName(), snapInfo.getBucketName());
+          OmBucketInfo bucketInfo = ozoneManager.getMetadataManager()
+              .getBucketTable().get(dbBucketKey);
+
+          if (bucketInfo == null) {
+            throw new OMException("Bucket " + snapInfo.getBucketName() +
+                " is not found", BUCKET_NOT_FOUND);
+          }
+
+          //TODO: [SNAPSHOT] Add lock to deletedTable and Active DB.
+          SnapshotInfo previousSnapshot = getPreviousSnapshot(snapInfo);
+          Table<String, OmKeyInfo> previousKeyTable = null;
+          OmSnapshot omPreviousSnapshot = null;
+
+          // Handle case when the deleted snapshot is the first snapshot.
+          // Move deleted keys to activeDB's deletedKeyTable
+          if (previousSnapshot != null) {
+            omPreviousSnapshot = (OmSnapshot) omSnapshotManager
+                .checkForSnapshot(previousSnapshot.getVolumeName(),
+                    previousSnapshot.getBucketName(),
+                    getSnapshotPrefix(previousSnapshot.getName()));
+
+            previousKeyTable = omPreviousSnapshot
+                .getMetadataManager().getKeyTable(bucketInfo.getBucketLayout());
+          }
+
+          // Move key to either next non deleted snapshot's snapshotDeletedTable
+          // or move to active object store deleted table
+
+          List<SnapshotMoveKeyInfos> toActiveDBList = new ArrayList<>();
+          List<SnapshotMoveKeyInfos> toNextDBList = new ArrayList<>();
+
+          try (TableIterator<String, ? extends Table.KeyValue<String,
+              RepeatedOmKeyInfo>> deletedIterator = snapshotDeletedTable
+              .iterator()) {
+
+            String snapshotBucketKey = dbBucketKey + OzoneConsts.OM_KEY_PREFIX;
+            iterator.seek(snapshotBucketKey);
+
+            while (deletedIterator.hasNext()) {
+              Table.KeyValue<String, RepeatedOmKeyInfo>
+                  deletedKeyValue = deletedIterator.next();
+              String deletedKey = deletedKeyValue.getKey();
+
+              // Exit if it is out of the bucket scope.
+              if (!deletedKey.contains(snapshotBucketKey)) {
+                break;
+              }
+
+              RepeatedOmKeyInfo repeatedOmKeyInfo = deletedKeyValue.getValue();
+
+              SnapshotMoveKeyInfos.Builder toActiveDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+              SnapshotMoveKeyInfos.Builder toNextDb = SnapshotMoveKeyInfos
+                  .newBuilder()
+                  .setKey(deletedKey);
+
+              for (OmKeyInfo keyInfo: repeatedOmKeyInfo.getOmKeyInfoList()) {
+                splitRepeatedOmKeyInfo(toActiveDb, toNextDb,
+                    keyInfo, previousKeyTable);
+              }
+
+              toActiveDBList.add(toActiveDb.build());
+              toNextDBList.add(toNextDb.build());
+
+            }
+            // Submit Move request to OM.
+            submitSnapshotMoveDeletedKeys(snapInfo, toActiveDBList,
+                toNextDBList);
+            snapshotLimit--;

Review Comment:
   Thanks for pointing that out. Yes, we should be, but I'm putting it on another patch. 



-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/WithObjectID.java:
##########
@@ -61,10 +61,6 @@ public long getUpdateID() {
    * @param obId - long
    */
   public void setObjectID(long obId) {
-    if (this.objectID != 0) {

Review Comment:
   Or adjust the condition to make `obId` equals `OBJECT_ID_DEFAULT` the only exception:
   
   ```
   if (this.objectID != 0 && obId != OBJECT_ID_DEFAULT) {
   ```



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

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

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


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


[GitHub] [ozone] aswinshakil commented on pull request #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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

   I'm disabling the tests for this PR and will update it in another patch, there are [current PR](https://github.com/apache/ozone/pull/4280) and my upcoming [patch](https://issues.apache.org/jira/browse/HDDS-7883) that would break the tests when merging to master. 


-- 
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 #4244: HDDS-7740. [Snapshot] Implement SnapshotDeletingService

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


##########
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/snapshot/OMSnapshotMoveDeletedKeysRequest.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.ozone.om.request.snapshot;
+
+import org.apache.hadoop.ozone.om.OmSnapshot;
+import org.apache.hadoop.ozone.om.OmSnapshotManager;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.SnapshotChainManager;
+import org.apache.hadoop.ozone.om.helpers.SnapshotInfo;
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
+import org.apache.hadoop.ozone.om.request.OMClientRequest;
+import org.apache.hadoop.ozone.om.request.util.OmResponseUtil;
+import org.apache.hadoop.ozone.om.response.OMClientResponse;
+import org.apache.hadoop.ozone.om.response.snapshot.OMSnapshotMoveDeletedKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveKeyInfos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SnapshotMoveDeletedKeysRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.List;
+
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+
+/**
+ * Handles OMSnapshotMoveDeletedKeys Request.
+ */
+public class OMSnapshotMoveDeletedKeysRequest extends OMClientRequest {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMSnapshotMoveDeletedKeysRequest.class);
+
+  public OMSnapshotMoveDeletedKeysRequest(OMRequest omRequest) {
+    super(omRequest);
+  }
+
+  @Override
+  public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
+      long trxnLogIndex, OzoneManagerDoubleBufferHelper omDoubleBufferHelper) {
+    OmSnapshotManager omSnapshotManager = ozoneManager.getOmSnapshotManager();
+    SnapshotChainManager snapshotChainManager =
+        ozoneManager.getSnapshotChainManager();
+
+    SnapshotMoveDeletedKeysRequest moveDeletedKeysRequest =
+        getOmRequest().getSnapshotMoveDeletedKeysRequest();
+    SnapshotInfo fromSnapshot = SnapshotInfo.getFromProtobuf(
+        moveDeletedKeysRequest.getFromSnapshot());
+
+    // If there is no Non-Deleted Snapshot move the
+    // keys to Active Object Store.
+    SnapshotInfo nextSnapshot = null;
+    OMClientResponse omClientResponse = null;
+    OzoneManagerProtocolProtos.OMResponse.Builder omResponse =
+        OmResponseUtil.getOMResponseBuilder(getOmRequest());
+    try {
+      OmSnapshot omFromSnapshot = (OmSnapshot) omSnapshotManager
+          .checkForSnapshot(fromSnapshot.getVolumeName(),
+              fromSnapshot.getBucketName(),
+              getSnapshotPrefix(fromSnapshot.getName()));
+
+      nextSnapshot = getNextActiveSnapshot(fromSnapshot,
+          snapshotChainManager, omSnapshotManager);
+
+      // Get next non-deleted snapshot.
+      List<SnapshotMoveKeyInfos> nextDBKeysList =
+          moveDeletedKeysRequest.getNextDBKeysList();
+      List<SnapshotMoveKeyInfos> reclaimKeysList =
+          moveDeletedKeysRequest.getReclaimKeysList();
+
+      OmSnapshot omNextSnapshot = null;
+
+      if (nextSnapshot != null) {
+        omNextSnapshot = (OmSnapshot) omSnapshotManager
+            .checkForSnapshot(nextSnapshot.getVolumeName(),
+                nextSnapshot.getBucketName(),
+                getSnapshotPrefix(nextSnapshot.getName()));
+      }
+
+      omClientResponse = new OMSnapshotMoveDeletedKeysResponse(
+          omResponse.build(), omFromSnapshot, omNextSnapshot,
+          nextDBKeysList, reclaimKeysList);
+
+    } catch (IOException ex) {
+      omClientResponse = new OMSnapshotMoveDeletedKeysResponse(
+          createErrorOMResponse(omResponse, ex));
+    } finally {
+      addResponseToDoubleBuffer(trxnLogIndex, omClientResponse,
+          omDoubleBufferHelper);
+    }
+
+    return omClientResponse;
+  }
+
+  /**
+   * Get the next non deleted snapshot in the snapshot chain.
+   */
+  private SnapshotInfo getNextActiveSnapshot(SnapshotInfo snapInfo,
+      SnapshotChainManager chainManager, OmSnapshotManager omSnapshotManager)
+      throws IOException {
+    while (chainManager.hasNextPathSnapshot(snapInfo.getSnapshotPath(),

Review Comment:
   I think it might cause an infinite loop because `snapInfo` is not getting reset.



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