You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by "hemantk-12 (via GitHub)" <gi...@apache.org> on 2023/09/05 22:53:43 UTC

[GitHub] [ozone] hemantk-12 commented on a diff in pull request #5242: HDDS-9209. Re-enable check for backup compaction files pruning

hemantk-12 commented on code in PR #5242:
URL: https://github.com/apache/ozone/pull/5242#discussion_r1316475043


##########
hadoop-hdds/rocksdb-checkpoint-differ/src/main/java/org/apache/ozone/rocksdiff/RocksDBCheckpointDiffer.java:
##########
@@ -185,6 +186,7 @@ public class RocksDBCheckpointDiffer implements AutoCloseable,
   private ColumnFamilyHandle snapshotInfoTableCFHandle;
   private final AtomicInteger tarballRequestCount;
   private final String dagPruningServiceName = "CompactionDagPruningService";
+  private AtomicBoolean suspended;

Review Comment:
   nit: this can be private if I'm not wrong.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java:
##########
@@ -0,0 +1,722 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.hdds.utils.TransactionInfo;
+import org.apache.hadoop.hdds.utils.db.RDBCheckpointUtils;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException;
+import org.apache.hadoop.ozone.om.exceptions.OMNotLeaderException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+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.snapshot.ReferenceCounted;
+import org.apache.hadoop.ozone.om.snapshot.SnapshotCache;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffReportOzone;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.Timeout;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPath;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.TestOzoneManagerHAWithData.createKey;
+import static org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse.JobStatus.DONE;
+
+/**
+ * Tests snapshot background services.
+ */
+@Timeout(5000)
+public class TestSnapshotBackgroundServices {
+
+  private MiniOzoneHAClusterImpl cluster = null;
+  private ObjectStore objectStore;
+  private OzoneBucket ozoneBucket;
+  private String volumeName;
+  private String bucketName;
+
+  private static final long SNAPSHOT_THRESHOLD = 50;
+  private static final int LOG_PURGE_GAP = 50;
+  // This test depends on direct RocksDB checks that are easier done with OBS
+  // buckets.
+  private static final BucketLayout TEST_BUCKET_LAYOUT =
+      BucketLayout.OBJECT_STORE;
+  private static final String SNAPSHOT_NAME_PREFIX = "snapshot";
+  private OzoneClient client;
+
+  /**
+   * Create a MiniOzoneCluster for testing. The cluster initially has one
+   * inactive OM. So at the start of the cluster, there will be 2 active and 1
+   * inactive OM.
+   */
+  @BeforeEach
+  public void init(TestInfo testInfo) throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omServiceId = "om-service-test1";
+    conf.setInt(OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_GAP, LOG_PURGE_GAP);
+    conf.setStorageSize(OMConfigKeys.OZONE_OM_RATIS_SEGMENT_SIZE_KEY, 16,
+        StorageUnit.KB);
+    conf.setStorageSize(OMConfigKeys.
+        OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY, 16, StorageUnit.KB);
+    if ("testSSTFilteringBackgroundService".equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+    }
+    if ("testCompactionLogBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+    }
+    if ("testBackupCompactionFilesPruningBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+      conf.setTimeDuration(
+          OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL, 1,
+          TimeUnit.SECONDS);
+    }
+    if ("testSnapshotAndKeyDeletionBackgroundServices"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+      conf.setTimeDuration(
+          OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL, 3,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL, 3,
+          TimeUnit.SECONDS);
+    }
+    conf.setLong(
+        OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_AUTO_TRIGGER_THRESHOLD_KEY,
+        SNAPSHOT_THRESHOLD);
+    int numOfOMs = 3;
+    cluster = (MiniOzoneHAClusterImpl) MiniOzoneCluster.newOMHABuilder(conf)
+        .setClusterId(clusterId)
+        .setScmId(scmId)
+        .setOMServiceId("om-service-test1")
+        .setNumOfOzoneManagers(numOfOMs)
+        .setNumOfActiveOMs(2)
+        .build();
+    if ("testBackupCompactionFilesPruningBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      cluster.
+          getOzoneManagersList()
+          .forEach(
+              TestSnapshotBackgroundServices
+                  ::suspendBackupCompactionFilesPruning);
+    }
+    cluster.waitForClusterToBeReady();
+    client = OzoneClientFactory.getRpcClient(omServiceId, conf);
+    objectStore = client.getObjectStore();
+
+    volumeName = "volume" + RandomStringUtils.randomNumeric(5);
+    bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
+
+    VolumeArgs createVolumeArgs = VolumeArgs.newBuilder()
+        .setOwner("user" + RandomStringUtils.randomNumeric(5))
+        .setAdmin("admin" + RandomStringUtils.randomNumeric(5))
+        .build();
+
+    objectStore.createVolume(volumeName, createVolumeArgs);
+    OzoneVolume retVolumeinfo = objectStore.getVolume(volumeName);
+
+    retVolumeinfo.createBucket(bucketName,
+        BucketArgs.newBuilder().setBucketLayout(TEST_BUCKET_LAYOUT).build());
+    ozoneBucket = retVolumeinfo.getBucket(bucketName);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterEach
+  public void shutdown() {
+    IOUtils.closeQuietly(client);
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  @DisplayName("testSnapshotAndKeyDeletionBackgroundServices")
+  @SuppressWarnings("methodlength")
+  public void testSnapshotAndKeyDeletionBackgroundServices()
+      throws IOException, InterruptedException, TimeoutException {
+    OzoneManager leaderOM = getLeaderOM();
+    OzoneManager followerOM = getInactiveFollowerOM(leaderOM);
+
+    createSnapshotsEachWithNewKeys(leaderOM);
+
+    startInactiveFollower(leaderOM, followerOM, () -> {
+    });
+
+    // Read & Write after snapshot installed.
+    List<String> newKeys = writeKeys(1);
+    readKeys(newKeys);
+
+    OzoneManager newLeaderOM =
+        getNewLeader(leaderOM, followerOM.getOMNodeId(), followerOM);
+    OzoneManager newFollowerOM =
+        cluster.getOzoneManager(leaderOM.getOMNodeId());
+    Assertions.assertEquals(leaderOM, newFollowerOM);
+
+    SnapshotInfo newSnapshot = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+
+      /*

Review Comment:
   ```suggestion
       /*
   ```



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java:
##########
@@ -0,0 +1,722 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.hdds.utils.TransactionInfo;
+import org.apache.hadoop.hdds.utils.db.RDBCheckpointUtils;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException;
+import org.apache.hadoop.ozone.om.exceptions.OMNotLeaderException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+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.snapshot.ReferenceCounted;
+import org.apache.hadoop.ozone.om.snapshot.SnapshotCache;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffReportOzone;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.Timeout;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPath;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.TestOzoneManagerHAWithData.createKey;
+import static org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse.JobStatus.DONE;
+
+/**
+ * Tests snapshot background services.
+ */
+@Timeout(5000)
+public class TestSnapshotBackgroundServices {
+
+  private MiniOzoneHAClusterImpl cluster = null;
+  private ObjectStore objectStore;
+  private OzoneBucket ozoneBucket;
+  private String volumeName;
+  private String bucketName;
+
+  private static final long SNAPSHOT_THRESHOLD = 50;
+  private static final int LOG_PURGE_GAP = 50;
+  // This test depends on direct RocksDB checks that are easier done with OBS
+  // buckets.
+  private static final BucketLayout TEST_BUCKET_LAYOUT =
+      BucketLayout.OBJECT_STORE;
+  private static final String SNAPSHOT_NAME_PREFIX = "snapshot";
+  private OzoneClient client;
+
+  /**
+   * Create a MiniOzoneCluster for testing. The cluster initially has one
+   * inactive OM. So at the start of the cluster, there will be 2 active and 1
+   * inactive OM.
+   */
+  @BeforeEach
+  public void init(TestInfo testInfo) throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omServiceId = "om-service-test1";
+    conf.setInt(OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_GAP, LOG_PURGE_GAP);
+    conf.setStorageSize(OMConfigKeys.OZONE_OM_RATIS_SEGMENT_SIZE_KEY, 16,
+        StorageUnit.KB);
+    conf.setStorageSize(OMConfigKeys.
+        OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY, 16, StorageUnit.KB);
+    if ("testSSTFilteringBackgroundService".equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+    }
+    if ("testCompactionLogBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+    }
+    if ("testBackupCompactionFilesPruningBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+      conf.setTimeDuration(
+          OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL, 1,
+          TimeUnit.SECONDS);
+    }
+    if ("testSnapshotAndKeyDeletionBackgroundServices"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+      conf.setTimeDuration(
+          OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL, 3,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL, 3,
+          TimeUnit.SECONDS);
+    }
+    conf.setLong(
+        OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_AUTO_TRIGGER_THRESHOLD_KEY,
+        SNAPSHOT_THRESHOLD);
+    int numOfOMs = 3;
+    cluster = (MiniOzoneHAClusterImpl) MiniOzoneCluster.newOMHABuilder(conf)
+        .setClusterId(clusterId)
+        .setScmId(scmId)
+        .setOMServiceId("om-service-test1")
+        .setNumOfOzoneManagers(numOfOMs)
+        .setNumOfActiveOMs(2)
+        .build();
+    if ("testBackupCompactionFilesPruningBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      cluster.
+          getOzoneManagersList()
+          .forEach(
+              TestSnapshotBackgroundServices
+                  ::suspendBackupCompactionFilesPruning);
+    }
+    cluster.waitForClusterToBeReady();
+    client = OzoneClientFactory.getRpcClient(omServiceId, conf);
+    objectStore = client.getObjectStore();
+
+    volumeName = "volume" + RandomStringUtils.randomNumeric(5);
+    bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
+
+    VolumeArgs createVolumeArgs = VolumeArgs.newBuilder()
+        .setOwner("user" + RandomStringUtils.randomNumeric(5))
+        .setAdmin("admin" + RandomStringUtils.randomNumeric(5))
+        .build();
+
+    objectStore.createVolume(volumeName, createVolumeArgs);
+    OzoneVolume retVolumeinfo = objectStore.getVolume(volumeName);
+
+    retVolumeinfo.createBucket(bucketName,
+        BucketArgs.newBuilder().setBucketLayout(TEST_BUCKET_LAYOUT).build());
+    ozoneBucket = retVolumeinfo.getBucket(bucketName);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterEach
+  public void shutdown() {
+    IOUtils.closeQuietly(client);
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  @DisplayName("testSnapshotAndKeyDeletionBackgroundServices")
+  @SuppressWarnings("methodlength")
+  public void testSnapshotAndKeyDeletionBackgroundServices()
+      throws IOException, InterruptedException, TimeoutException {
+    OzoneManager leaderOM = getLeaderOM();
+    OzoneManager followerOM = getInactiveFollowerOM(leaderOM);
+
+    createSnapshotsEachWithNewKeys(leaderOM);
+
+    startInactiveFollower(leaderOM, followerOM, () -> {
+    });
+
+    // Read & Write after snapshot installed.
+    List<String> newKeys = writeKeys(1);
+    readKeys(newKeys);
+
+    OzoneManager newLeaderOM =
+        getNewLeader(leaderOM, followerOM.getOMNodeId(), followerOM);
+    OzoneManager newFollowerOM =
+        cluster.getOzoneManager(leaderOM.getOMNodeId());
+    Assertions.assertEquals(leaderOM, newFollowerOM);
+
+    SnapshotInfo newSnapshot = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+
+      /*
+      Check whether newly created key data is reclaimed
+      create key a
+      create snapshot b
+      delete key a
+      create snapshot c
+      assert that a is in c's deleted table
+      create snapshot d
+      delete snapshot c
+      wait until key a appears in deleted table of d.
+    */
+    // create key a
+    String keyNameA = writeKeys(1).get(0);
+    String keyA = OM_KEY_PREFIX + ozoneBucket.getVolumeName() +
+        OM_KEY_PREFIX + ozoneBucket.getName() +
+        OM_KEY_PREFIX + keyNameA;
+    Table<String, OmKeyInfo> omKeyInfoTable = newLeaderOM
+        .getMetadataManager()
+        .getKeyTable(ozoneBucket.getBucketLayout());
+    OmKeyInfo keyInfoA = omKeyInfoTable.get(keyA);
+    Assertions.assertNotNull(keyInfoA);
+
+    // create snapshot b
+    SnapshotInfo snapshotInfoB = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+    Assertions.assertNotNull(snapshotInfoB);
+
+    // delete key a
+    ozoneBucket.deleteKey(keyNameA);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return Objects.isNull(omKeyInfoTable.get(keyA));
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }, 1000, 10000);
+
+    // create snapshot c
+    SnapshotInfo snapshotInfoC = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+
+    // get snapshot c
+    OmSnapshot snapC;
+    try (ReferenceCounted<IOmMetadataReader, SnapshotCache> rcC = newLeaderOM
+        .getOmSnapshotManager()
+        .checkForSnapshot(volumeName, bucketName,
+            getSnapshotPrefix(snapshotInfoC.getName()), true)) {
+      Assertions.assertNotNull(rcC);
+      snapC = (OmSnapshot) rcC.get();
+    }
+
+    // assert that key a is in snapshot c's deleted table
+    GenericTestUtils.waitFor(() -> {
+      try (TableIterator<String, ? extends Table.KeyValue<String,

Review Comment:
   1. What's the reason to iterate over the table? Can we use `get(keyA)`, `getSkipCache(keyA)` or `getIfExist(keyA)`?  Something like below?
   
   ```
       GenericTestUtils.waitFor(() -> {
         try {
           return snapC.getMetadataManager().getDeletedTable().getIfExist(keyA)
               != null;
         } catch (IOException e) {
           Assertions.fail();
           return false;
         }
       }, 1000, 10000);
   ```
   
   2. This repetitive code and similar code is written at 329-343. May be something like if you don't care about the value type..
   ```
     GenericTestUtils.waitFor(() ->
           checkForKeyInTable(snapC.getMetadataManager().getDeletedTable(), keyA),
       1000, 10000);  
     ...
     ...
     private <V> boolean checkForKeyInTable(Table<String, V> table,
                                            String key) {
       try (TableIterator<String, ? extends Table.KeyValue<String, V>> iterator
                = table.iterator()) {
         while (iterator.hasNext()) {
           Table.KeyValue<String, V> next = iterator.next();
           if (next.getKey().contains(key)) {
             return true;
           }
         }
         return false;
       } catch (IOException e) {
         return false;
       }
     }
   ```



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java:
##########
@@ -0,0 +1,722 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.hdds.utils.TransactionInfo;
+import org.apache.hadoop.hdds.utils.db.RDBCheckpointUtils;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException;
+import org.apache.hadoop.ozone.om.exceptions.OMNotLeaderException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+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.snapshot.ReferenceCounted;
+import org.apache.hadoop.ozone.om.snapshot.SnapshotCache;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffReportOzone;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.Timeout;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPath;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.TestOzoneManagerHAWithData.createKey;
+import static org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse.JobStatus.DONE;
+
+/**
+ * Tests snapshot background services.
+ */
+@Timeout(5000)
+public class TestSnapshotBackgroundServices {
+
+  private MiniOzoneHAClusterImpl cluster = null;
+  private ObjectStore objectStore;
+  private OzoneBucket ozoneBucket;
+  private String volumeName;
+  private String bucketName;
+
+  private static final long SNAPSHOT_THRESHOLD = 50;
+  private static final int LOG_PURGE_GAP = 50;
+  // This test depends on direct RocksDB checks that are easier done with OBS
+  // buckets.
+  private static final BucketLayout TEST_BUCKET_LAYOUT =
+      BucketLayout.OBJECT_STORE;
+  private static final String SNAPSHOT_NAME_PREFIX = "snapshot";
+  private OzoneClient client;
+
+  /**
+   * Create a MiniOzoneCluster for testing. The cluster initially has one
+   * inactive OM. So at the start of the cluster, there will be 2 active and 1
+   * inactive OM.
+   */
+  @BeforeEach
+  public void init(TestInfo testInfo) throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omServiceId = "om-service-test1";
+    conf.setInt(OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_GAP, LOG_PURGE_GAP);
+    conf.setStorageSize(OMConfigKeys.OZONE_OM_RATIS_SEGMENT_SIZE_KEY, 16,
+        StorageUnit.KB);
+    conf.setStorageSize(OMConfigKeys.
+        OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY, 16, StorageUnit.KB);
+    if ("testSSTFilteringBackgroundService".equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+    }
+    if ("testCompactionLogBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+    }
+    if ("testBackupCompactionFilesPruningBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+      conf.setTimeDuration(
+          OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL, 1,
+          TimeUnit.SECONDS);
+    }
+    if ("testSnapshotAndKeyDeletionBackgroundServices"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+      conf.setTimeDuration(
+          OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL, 3,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL, 3,
+          TimeUnit.SECONDS);
+    }
+    conf.setLong(
+        OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_AUTO_TRIGGER_THRESHOLD_KEY,
+        SNAPSHOT_THRESHOLD);
+    int numOfOMs = 3;
+    cluster = (MiniOzoneHAClusterImpl) MiniOzoneCluster.newOMHABuilder(conf)
+        .setClusterId(clusterId)
+        .setScmId(scmId)
+        .setOMServiceId("om-service-test1")
+        .setNumOfOzoneManagers(numOfOMs)
+        .setNumOfActiveOMs(2)
+        .build();
+    if ("testBackupCompactionFilesPruningBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      cluster.
+          getOzoneManagersList()
+          .forEach(
+              TestSnapshotBackgroundServices
+                  ::suspendBackupCompactionFilesPruning);
+    }
+    cluster.waitForClusterToBeReady();
+    client = OzoneClientFactory.getRpcClient(omServiceId, conf);
+    objectStore = client.getObjectStore();
+
+    volumeName = "volume" + RandomStringUtils.randomNumeric(5);
+    bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
+
+    VolumeArgs createVolumeArgs = VolumeArgs.newBuilder()
+        .setOwner("user" + RandomStringUtils.randomNumeric(5))
+        .setAdmin("admin" + RandomStringUtils.randomNumeric(5))
+        .build();
+
+    objectStore.createVolume(volumeName, createVolumeArgs);
+    OzoneVolume retVolumeinfo = objectStore.getVolume(volumeName);
+
+    retVolumeinfo.createBucket(bucketName,
+        BucketArgs.newBuilder().setBucketLayout(TEST_BUCKET_LAYOUT).build());
+    ozoneBucket = retVolumeinfo.getBucket(bucketName);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterEach
+  public void shutdown() {
+    IOUtils.closeQuietly(client);
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  @DisplayName("testSnapshotAndKeyDeletionBackgroundServices")
+  @SuppressWarnings("methodlength")
+  public void testSnapshotAndKeyDeletionBackgroundServices()
+      throws IOException, InterruptedException, TimeoutException {
+    OzoneManager leaderOM = getLeaderOM();
+    OzoneManager followerOM = getInactiveFollowerOM(leaderOM);
+
+    createSnapshotsEachWithNewKeys(leaderOM);
+
+    startInactiveFollower(leaderOM, followerOM, () -> {
+    });
+
+    // Read & Write after snapshot installed.
+    List<String> newKeys = writeKeys(1);
+    readKeys(newKeys);
+
+    OzoneManager newLeaderOM =
+        getNewLeader(leaderOM, followerOM.getOMNodeId(), followerOM);
+    OzoneManager newFollowerOM =
+        cluster.getOzoneManager(leaderOM.getOMNodeId());
+    Assertions.assertEquals(leaderOM, newFollowerOM);
+
+    SnapshotInfo newSnapshot = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+
+      /*
+      Check whether newly created key data is reclaimed
+      create key a
+      create snapshot b
+      delete key a
+      create snapshot c
+      assert that a is in c's deleted table
+      create snapshot d
+      delete snapshot c
+      wait until key a appears in deleted table of d.
+    */
+    // create key a
+    String keyNameA = writeKeys(1).get(0);
+    String keyA = OM_KEY_PREFIX + ozoneBucket.getVolumeName() +
+        OM_KEY_PREFIX + ozoneBucket.getName() +
+        OM_KEY_PREFIX + keyNameA;
+    Table<String, OmKeyInfo> omKeyInfoTable = newLeaderOM
+        .getMetadataManager()
+        .getKeyTable(ozoneBucket.getBucketLayout());
+    OmKeyInfo keyInfoA = omKeyInfoTable.get(keyA);
+    Assertions.assertNotNull(keyInfoA);
+
+    // create snapshot b
+    SnapshotInfo snapshotInfoB = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+    Assertions.assertNotNull(snapshotInfoB);
+
+    // delete key a
+    ozoneBucket.deleteKey(keyNameA);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return Objects.isNull(omKeyInfoTable.get(keyA));
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }, 1000, 10000);
+
+    // create snapshot c
+    SnapshotInfo snapshotInfoC = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+
+    // get snapshot c
+    OmSnapshot snapC;
+    try (ReferenceCounted<IOmMetadataReader, SnapshotCache> rcC = newLeaderOM
+        .getOmSnapshotManager()
+        .checkForSnapshot(volumeName, bucketName,
+            getSnapshotPrefix(snapshotInfoC.getName()), true)) {
+      Assertions.assertNotNull(rcC);
+      snapC = (OmSnapshot) rcC.get();
+    }
+
+    // assert that key a is in snapshot c's deleted table
+    GenericTestUtils.waitFor(() -> {
+      try (TableIterator<String, ? extends Table.KeyValue<String,
+          RepeatedOmKeyInfo>> iterator =
+               snapC.getMetadataManager().getDeletedTable().iterator()) {
+        while (iterator.hasNext()) {
+          if (iterator.next().getKey().contains(keyA)) {
+            return true;
+          }
+        }
+
+        return false;
+      } catch (IOException e) {
+        Assertions.fail();
+        return false;
+      }
+    }, 1000, 10000);
+
+    // create snapshot d
+    SnapshotInfo snapshotInfoD = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+
+    // delete snapshot c
+    client.getObjectStore()
+        .deleteSnapshot(volumeName, bucketName, snapshotInfoC.getName());
+
+    GenericTestUtils.waitFor(() -> {
+      Table<String, SnapshotInfo> snapshotInfoTable =

Review Comment:
   Similar to previous comment, snapshot check can be moved to a helper function.
   
   If helper function is not nicer, may be use LambdaTestUtils#await(). If I'm not wrong it catches  any checked Exception and convert them to RuntimeException. You don't have to add catch everywhere if you are simply re-throwing them as `RuntimeException`.



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestSnapshotBackgroundServices.java:
##########
@@ -0,0 +1,722 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.conf.StorageUnit;
+import org.apache.hadoop.hdds.utils.IOUtils;
+import org.apache.hadoop.hdds.utils.TransactionInfo;
+import org.apache.hadoop.hdds.utils.db.RDBCheckpointUtils;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl;
+import org.apache.hadoop.ozone.client.BucketArgs;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.client.VolumeArgs;
+import org.apache.hadoop.ozone.client.io.OzoneInputStream;
+import org.apache.hadoop.ozone.om.exceptions.OMLeaderNotReadyException;
+import org.apache.hadoop.ozone.om.exceptions.OMNotLeaderException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+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.snapshot.ReferenceCounted;
+import org.apache.hadoop.ozone.om.snapshot.SnapshotCache;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffReportOzone;
+import org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse;
+import org.apache.ozone.test.GenericTestUtils;
+import org.apache.ratis.server.protocol.TermIndex;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.DisplayName;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.TestInfo;
+import org.junit.jupiter.api.Timeout;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.OzoneConsts.OM_KEY_PREFIX;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPath;
+import static org.apache.hadoop.ozone.om.OmSnapshotManager.getSnapshotPrefix;
+import static org.apache.hadoop.ozone.om.TestOzoneManagerHAWithData.createKey;
+import static org.apache.hadoop.ozone.snapshot.SnapshotDiffResponse.JobStatus.DONE;
+
+/**
+ * Tests snapshot background services.
+ */
+@Timeout(5000)
+public class TestSnapshotBackgroundServices {
+
+  private MiniOzoneHAClusterImpl cluster = null;
+  private ObjectStore objectStore;
+  private OzoneBucket ozoneBucket;
+  private String volumeName;
+  private String bucketName;
+
+  private static final long SNAPSHOT_THRESHOLD = 50;
+  private static final int LOG_PURGE_GAP = 50;
+  // This test depends on direct RocksDB checks that are easier done with OBS
+  // buckets.
+  private static final BucketLayout TEST_BUCKET_LAYOUT =
+      BucketLayout.OBJECT_STORE;
+  private static final String SNAPSHOT_NAME_PREFIX = "snapshot";
+  private OzoneClient client;
+
+  /**
+   * Create a MiniOzoneCluster for testing. The cluster initially has one
+   * inactive OM. So at the start of the cluster, there will be 2 active and 1
+   * inactive OM.
+   */
+  @BeforeEach
+  public void init(TestInfo testInfo) throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omServiceId = "om-service-test1";
+    conf.setInt(OMConfigKeys.OZONE_OM_RATIS_LOG_PURGE_GAP, LOG_PURGE_GAP);
+    conf.setStorageSize(OMConfigKeys.OZONE_OM_RATIS_SEGMENT_SIZE_KEY, 16,
+        StorageUnit.KB);
+    conf.setStorageSize(OMConfigKeys.
+        OZONE_OM_RATIS_SEGMENT_PREALLOCATED_SIZE_KEY, 16, StorageUnit.KB);
+    if ("testSSTFilteringBackgroundService".equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+    }
+    if ("testCompactionLogBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+    }
+    if ("testBackupCompactionFilesPruningBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+      conf.setTimeDuration(
+          OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL, 1,
+          TimeUnit.SECONDS);
+    }
+    if ("testSnapshotAndKeyDeletionBackgroundServices"
+        .equals(testInfo.getDisplayName())) {
+      conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_SNAPSHOT_DELETING_SERVICE_INTERVAL, 1,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_OM_SNAPSHOT_COMPACTION_DAG_MAX_TIME_ALLOWED, 1,
+          TimeUnit.MILLISECONDS);
+      conf.setTimeDuration(
+          OZONE_OM_SNAPSHOT_COMPACTION_DAG_PRUNE_DAEMON_RUN_INTERVAL, 3,
+          TimeUnit.SECONDS);
+      conf.setTimeDuration(OZONE_SNAPSHOT_SST_FILTERING_SERVICE_INTERVAL, 3,
+          TimeUnit.SECONDS);
+    }
+    conf.setLong(
+        OMConfigKeys.OZONE_OM_RATIS_SNAPSHOT_AUTO_TRIGGER_THRESHOLD_KEY,
+        SNAPSHOT_THRESHOLD);
+    int numOfOMs = 3;
+    cluster = (MiniOzoneHAClusterImpl) MiniOzoneCluster.newOMHABuilder(conf)
+        .setClusterId(clusterId)
+        .setScmId(scmId)
+        .setOMServiceId("om-service-test1")
+        .setNumOfOzoneManagers(numOfOMs)
+        .setNumOfActiveOMs(2)
+        .build();
+    if ("testBackupCompactionFilesPruningBackgroundService"
+        .equals(testInfo.getDisplayName())) {
+      cluster.
+          getOzoneManagersList()
+          .forEach(
+              TestSnapshotBackgroundServices
+                  ::suspendBackupCompactionFilesPruning);
+    }
+    cluster.waitForClusterToBeReady();
+    client = OzoneClientFactory.getRpcClient(omServiceId, conf);
+    objectStore = client.getObjectStore();
+
+    volumeName = "volume" + RandomStringUtils.randomNumeric(5);
+    bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
+
+    VolumeArgs createVolumeArgs = VolumeArgs.newBuilder()
+        .setOwner("user" + RandomStringUtils.randomNumeric(5))
+        .setAdmin("admin" + RandomStringUtils.randomNumeric(5))
+        .build();
+
+    objectStore.createVolume(volumeName, createVolumeArgs);
+    OzoneVolume retVolumeinfo = objectStore.getVolume(volumeName);
+
+    retVolumeinfo.createBucket(bucketName,
+        BucketArgs.newBuilder().setBucketLayout(TEST_BUCKET_LAYOUT).build());
+    ozoneBucket = retVolumeinfo.getBucket(bucketName);
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @AfterEach
+  public void shutdown() {
+    IOUtils.closeQuietly(client);
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  @DisplayName("testSnapshotAndKeyDeletionBackgroundServices")
+  @SuppressWarnings("methodlength")
+  public void testSnapshotAndKeyDeletionBackgroundServices()
+      throws IOException, InterruptedException, TimeoutException {
+    OzoneManager leaderOM = getLeaderOM();
+    OzoneManager followerOM = getInactiveFollowerOM(leaderOM);
+
+    createSnapshotsEachWithNewKeys(leaderOM);
+
+    startInactiveFollower(leaderOM, followerOM, () -> {
+    });
+
+    // Read & Write after snapshot installed.
+    List<String> newKeys = writeKeys(1);
+    readKeys(newKeys);
+
+    OzoneManager newLeaderOM =
+        getNewLeader(leaderOM, followerOM.getOMNodeId(), followerOM);
+    OzoneManager newFollowerOM =
+        cluster.getOzoneManager(leaderOM.getOMNodeId());
+    Assertions.assertEquals(leaderOM, newFollowerOM);
+
+    SnapshotInfo newSnapshot = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+
+      /*
+      Check whether newly created key data is reclaimed
+      create key a
+      create snapshot b
+      delete key a
+      create snapshot c
+      assert that a is in c's deleted table
+      create snapshot d
+      delete snapshot c
+      wait until key a appears in deleted table of d.
+    */
+    // create key a
+    String keyNameA = writeKeys(1).get(0);
+    String keyA = OM_KEY_PREFIX + ozoneBucket.getVolumeName() +
+        OM_KEY_PREFIX + ozoneBucket.getName() +
+        OM_KEY_PREFIX + keyNameA;
+    Table<String, OmKeyInfo> omKeyInfoTable = newLeaderOM
+        .getMetadataManager()
+        .getKeyTable(ozoneBucket.getBucketLayout());
+    OmKeyInfo keyInfoA = omKeyInfoTable.get(keyA);
+    Assertions.assertNotNull(keyInfoA);
+
+    // create snapshot b
+    SnapshotInfo snapshotInfoB = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+    Assertions.assertNotNull(snapshotInfoB);
+
+    // delete key a
+    ozoneBucket.deleteKey(keyNameA);
+
+    GenericTestUtils.waitFor(() -> {
+      try {
+        return Objects.isNull(omKeyInfoTable.get(keyA));
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }, 1000, 10000);
+
+    // create snapshot c
+    SnapshotInfo snapshotInfoC = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+
+    // get snapshot c
+    OmSnapshot snapC;
+    try (ReferenceCounted<IOmMetadataReader, SnapshotCache> rcC = newLeaderOM
+        .getOmSnapshotManager()
+        .checkForSnapshot(volumeName, bucketName,
+            getSnapshotPrefix(snapshotInfoC.getName()), true)) {
+      Assertions.assertNotNull(rcC);
+      snapC = (OmSnapshot) rcC.get();
+    }
+
+    // assert that key a is in snapshot c's deleted table
+    GenericTestUtils.waitFor(() -> {
+      try (TableIterator<String, ? extends Table.KeyValue<String,
+          RepeatedOmKeyInfo>> iterator =
+               snapC.getMetadataManager().getDeletedTable().iterator()) {
+        while (iterator.hasNext()) {
+          if (iterator.next().getKey().contains(keyA)) {
+            return true;
+          }
+        }
+
+        return false;
+      } catch (IOException e) {
+        Assertions.fail();
+        return false;
+      }
+    }, 1000, 10000);
+
+    // create snapshot d
+    SnapshotInfo snapshotInfoD = createOzoneSnapshot(newLeaderOM,
+        SNAPSHOT_NAME_PREFIX + RandomStringUtils.randomNumeric(5));
+
+    // delete snapshot c
+    client.getObjectStore()
+        .deleteSnapshot(volumeName, bucketName, snapshotInfoC.getName());
+
+    GenericTestUtils.waitFor(() -> {
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          newLeaderOM.getMetadataManager().getSnapshotInfoTable();
+      try {
+        return null == snapshotInfoTable.get(snapshotInfoC.getTableKey());
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }, 1000, 30000);
+
+    // get snapshot d
+    OmSnapshot snapD;
+    try (ReferenceCounted<IOmMetadataReader, SnapshotCache> rcD = newLeaderOM
+        .getOmSnapshotManager()
+        .checkForSnapshot(volumeName, bucketName,
+            getSnapshotPrefix(snapshotInfoD.getName()), true)) {
+      Assertions.assertNotNull(rcD);
+      snapD = (OmSnapshot) rcD.get();
+    }
+
+    // wait until key a appears in deleted table of snapshot d
+    GenericTestUtils.waitFor(() -> {
+      try (TableIterator<String, ? extends Table.KeyValue<String,
+          RepeatedOmKeyInfo>> iterator =
+               snapD.getMetadataManager().getDeletedTable().iterator()) {
+        while (iterator.hasNext()) {
+          Table.KeyValue<String, RepeatedOmKeyInfo> next = iterator.next();
+          if (next.getKey().contains(keyA)) {
+            return true;
+          }
+        }
+
+        return false;
+      } catch (IOException e) {
+        Assertions.fail();
+        return false;
+      }
+    }, 1000, 10000);
+
+    // Confirm entry for deleted snapshot removed from info table
+    client.getObjectStore()
+        .deleteSnapshot(volumeName, bucketName, newSnapshot.getName());
+    GenericTestUtils.waitFor(() -> {
+      Table<String, SnapshotInfo> snapshotInfoTable =
+          newLeaderOM.getMetadataManager().getSnapshotInfoTable();
+      try {
+        return null == snapshotInfoTable.get(newSnapshot.getTableKey());
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+    }, 1000, 10000);
+
+    confirmSnapDiffForTwoSnapshotsDifferingBySingleKey(
+        newLeaderOM);
+  }
+
+  private void startInactiveFollower(OzoneManager leaderOM,
+                                     OzoneManager followerOM,
+                                     Runnable actionAfterStarting)
+      throws IOException, TimeoutException, InterruptedException {
+    // Get the latest db checkpoint from the leader OM.
+    TransactionInfo transactionInfo =
+        TransactionInfo.readTransactionInfo(leaderOM.getMetadataManager());
+    TermIndex leaderOMTermIndex =
+        TermIndex.valueOf(transactionInfo.getTerm(),
+            transactionInfo.getTransactionIndex());
+    long leaderOMSnapshotIndex = leaderOMTermIndex.getIndex();
+
+    // Start the inactive OM. Checkpoint installation will happen spontaneously.
+    cluster.startInactiveOM(followerOM.getOMNodeId());
+    actionAfterStarting.run();
+
+    // The recently started OM should be lagging behind the leader OM.
+    // Wait & for follower to update transactions to leader snapshot index.
+    // Timeout error if follower does not load update within 10s
+    GenericTestUtils.waitFor(() ->
+        followerOM.getOmRatisServer().getLastAppliedTermIndex().getIndex()
+            >= leaderOMSnapshotIndex - 1, 100, 10000);
+
+    // Verify RPC server is running
+    GenericTestUtils.waitFor(followerOM::isOmRpcServerRunning, 100, 5000);
+  }
+
+  private void createSnapshotsEachWithNewKeys(OzoneManager ozoneManager)
+      throws IOException {
+    int keyIncrement = 10;
+    for (int snapshotCount = 0; snapshotCount < 10;
+         snapshotCount++) {
+      String snapshotName = SNAPSHOT_NAME_PREFIX + snapshotCount;
+      writeKeys(keyIncrement);
+      createOzoneSnapshot(ozoneManager, snapshotName);
+    }
+  }
+
+  private OzoneManager getInactiveFollowerOM(OzoneManager leaderOM) {
+    String followerNodeId = leaderOM.getPeerNodes().get(0).getNodeId();
+    if (cluster.isOMActive(followerNodeId)) {
+      followerNodeId = leaderOM.getPeerNodes().get(1).getNodeId();
+    }
+    return cluster.getOzoneManager(followerNodeId);
+  }
+
+  private OzoneManager getLeaderOM() {
+    String leaderOMNodeId = OmFailoverProxyUtil
+        .getFailoverProxyProvider(objectStore.getClientProxy())
+        .getCurrentProxyOMNodeId();
+    return cluster.getOzoneManager(leaderOMNodeId);
+  }
+
+  @Test
+  @DisplayName("testCompactionLogBackgroundService")
+  public void testCompactionLogBackgroundService()
+      throws IOException, InterruptedException, TimeoutException {
+    OzoneManager leaderOM = getLeaderOM();
+    OzoneManager followerOM = getInactiveFollowerOM(leaderOM);
+
+    createSnapshotsEachWithNewKeys(leaderOM);
+
+    startInactiveFollower(leaderOM, followerOM,
+        () -> suspendBackupCompactionFilesPruning(followerOM));
+
+    // Read & Write after snapshot installed.
+    List<String> newKeys = writeKeys(1);
+    readKeys(newKeys);
+
+    OzoneManager newLeaderOM =
+        getNewLeader(leaderOM, followerOM.getOMNodeId(), followerOM);
+    OzoneManager newFollowerOM =
+        cluster.getOzoneManager(leaderOM.getOMNodeId());
+    Assertions.assertEquals(leaderOM, newFollowerOM);
+
+    // Prepare baseline data for compaction logs
+    String currentCompactionLogPath = newLeaderOM
+        .getMetadataManager()
+        .getStore()
+        .getRocksDBCheckpointDiffer()
+        .getCurrentCompactionLogPath();
+    Assertions.assertNotNull(currentCompactionLogPath);
+    int lastIndex = currentCompactionLogPath.lastIndexOf(OM_KEY_PREFIX);
+    String compactionLogsPath = currentCompactionLogPath
+        .substring(0, lastIndex);
+    File compactionLogsDir = new File(compactionLogsPath);
+    Assertions.assertNotNull(compactionLogsDir);
+    File[] files = compactionLogsDir.listFiles();
+    Assertions.assertNotNull(files);
+    int numberOfLogFiles = files.length;
+    long contentLength;
+    Path currentCompactionLog = Paths.get(currentCompactionLogPath);
+    try (BufferedReader bufferedReader =
+             Files.newBufferedReader(currentCompactionLog)) {
+      contentLength = bufferedReader.lines()
+          .mapToLong(String::length)
+          .reduce(0L, Long::sum);
+    }
+
+    checkIfCompactionLogsGetAppendedByForcingCompaction(newLeaderOM,
+        compactionLogsDir, numberOfLogFiles, contentLength,
+        currentCompactionLog);
+
+    confirmSnapDiffForTwoSnapshotsDifferingBySingleKey(
+        newLeaderOM);
+  }
+
+  @Test
+  @DisplayName("testBackupCompactionFilesPruningBackgroundService")
+  public void testBackupCompactionFilesPruningBackgroundService()
+      throws IOException, InterruptedException, TimeoutException {
+    OzoneManager leaderOM = getLeaderOM();
+    OzoneManager followerOM = getInactiveFollowerOM(leaderOM);
+
+    startInactiveFollower(leaderOM, followerOM,
+        () -> suspendBackupCompactionFilesPruning(followerOM));
+
+    // Read & Write after snapshot installed.
+    List<String> newKeys = writeKeys(1);
+    readKeys(newKeys);
+
+    OzoneManager newLeaderOM =
+        getNewLeader(leaderOM, followerOM.getOMNodeId(), followerOM);
+    OzoneManager newFollowerOM =
+        cluster.getOzoneManager(leaderOM.getOMNodeId());
+    Assertions.assertEquals(leaderOM, newFollowerOM);
+
+    createSnapshotsEachWithNewKeys(newLeaderOM);
+
+    File sstBackupDir = getSstBackupDir(newLeaderOM);
+    File[] files = sstBackupDir.listFiles();
+    Assertions.assertNotNull(files);
+    int numberOfSstFiles = files.length;
+
+    resumeBackupCompactionFilesPruning(newLeaderOM);
+
+    checkIfCompactionBackupFilesWerePruned(sstBackupDir,
+        numberOfSstFiles);
+
+    confirmSnapDiffForTwoSnapshotsDifferingBySingleKey(newLeaderOM);
+  }
+
+  private static void resumeBackupCompactionFilesPruning(
+      OzoneManager ozoneManager) {
+    ozoneManager
+        .getMetadataManager()
+        .getStore()
+        .getRocksDBCheckpointDiffer()
+        .resume();
+  }
+
+  private static void suspendBackupCompactionFilesPruning(
+      OzoneManager ozoneManager) {
+    ozoneManager
+        .getMetadataManager()
+        .getStore()
+        .getRocksDBCheckpointDiffer()
+        .suspend();
+  }
+
+  @Test
+  @DisplayName("testSSTFilteringBackgroundService")
+  public void testSSTFilteringBackgroundService()
+      throws IOException, InterruptedException, TimeoutException {
+    OzoneManager leaderOM = getLeaderOM();
+    OzoneManager followerOM = getInactiveFollowerOM(leaderOM);
+
+    createSnapshotsEachWithNewKeys(leaderOM);
+
+    startInactiveFollower(leaderOM, followerOM, () -> {
+    });
+
+    // Read & Write after snapshot installed.
+    List<String> newKeys = writeKeys(1);
+    readKeys(newKeys);
+
+    OzoneManager newLeaderOM =
+        getNewLeader(leaderOM, followerOM.getOMNodeId(), followerOM);
+    OzoneManager newFollowerOM =
+        cluster.getOzoneManager(leaderOM.getOMNodeId());
+    Assertions.assertEquals(leaderOM, newFollowerOM);
+
+    checkIfSnapshotGetsProcessedBySFS(newLeaderOM);
+
+    confirmSnapDiffForTwoSnapshotsDifferingBySingleKey(
+        newLeaderOM);
+  }
+
+  private void confirmSnapDiffForTwoSnapshotsDifferingBySingleKey(
+      OzoneManager ozoneManager)
+      throws IOException, InterruptedException, TimeoutException {
+    String firstSnapshot = createOzoneSnapshot(ozoneManager,
+        TestSnapshotBackgroundServices.SNAPSHOT_NAME_PREFIX +
+            RandomStringUtils.randomNumeric(10)).getName();
+    String diffKey = writeKeys(1).get(0);
+    String secondSnapshot = createOzoneSnapshot(ozoneManager,
+        TestSnapshotBackgroundServices.SNAPSHOT_NAME_PREFIX +
+            RandomStringUtils.randomNumeric(10)).getName();
+    SnapshotDiffReportOzone diff = getSnapDiffReport(volumeName, bucketName,
+        firstSnapshot, secondSnapshot);
+    Assertions.assertEquals(Collections.singletonList(
+            SnapshotDiffReportOzone.getDiffReportEntry(
+                SnapshotDiffReport.DiffType.CREATE, diffKey, null)),
+        diff.getDiffList());
+  }
+
+  private static void checkIfCompactionBackupFilesWerePruned(File sstBackupDir,

Review Comment:
   nit: similar to class definition. 
   ```suggestion
     private static void checkIfCompactionBackupFilesWerePruned(
         File sstBackupDir,
         int numberOfSstFiles
     ) throws TimeoutException, InterruptedException {
   ```



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

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

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


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