Pārlūkot izejas kodu

HDFS-15524. Add edit log entry for Snapshot deletion GC thread snapshot deletion. (#2219)

bshashikant 4 gadi atpakaļ
vecāks
revīzija
15a76e8d65

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java

@@ -269,15 +269,15 @@ class FSDirSnapshotOp {
     // time of snapshot deletion
     final long now = Time.now();
     final INode.BlocksMapUpdateInfo collectedBlocks = deleteSnapshot(
-        fsd, snapshotManager, iip, snapshotName, now);
-    fsd.getEditLog().logDeleteSnapshot(snapshotRoot, snapshotName,
-        logRetryCache, now);
+        fsd, snapshotManager, iip, snapshotName, now, snapshotRoot,
+        logRetryCache);
     return collectedBlocks;
   }
 
   static INode.BlocksMapUpdateInfo deleteSnapshot(
       FSDirectory fsd, SnapshotManager snapshotManager, INodesInPath iip,
-      String snapshotName, long now) throws IOException {
+      String snapshotName, long now, String snapshotRoot, boolean logRetryCache)
+      throws IOException {
     INode.BlocksMapUpdateInfo collectedBlocks = new INode.BlocksMapUpdateInfo();
     ChunkedArrayList<INode> removedINodes = new ChunkedArrayList<>();
     INode.ReclaimContext context = new INode.ReclaimContext(
@@ -293,6 +293,8 @@ class FSDirSnapshotOp {
       fsd.writeUnlock();
     }
     removedINodes.clear();
+    fsd.getEditLog().logDeleteSnapshot(snapshotRoot, snapshotName,
+        logRetryCache, now);
     return collectedBlocks;
   }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -7235,7 +7235,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       final INodesInPath iip = dir.resolvePath(null, snapshotRoot, DirOp.WRITE);
       snapshotManager.assertMarkedAsDeleted(iip, snapshotName);
       blocksToBeDeleted = FSDirSnapshotOp.deleteSnapshot(
-          dir, snapshotManager, iip, snapshotName, now);
+          dir, snapshotManager, iip, snapshotName, now, snapshotRoot, false);
     } finally {
       writeUnlock(operationName, getLockReportInfoSupplier(rootPath));
     }

+ 50 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestOrderedSnapshotDeletionGc.java

@@ -21,6 +21,13 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.NNStorage;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
+import org.apache.hadoop.hdfs.util.Holder;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -28,26 +35,30 @@ import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.event.Level;
 
+import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Random;
+import java.util.EnumMap;
+import java.util.ArrayList;
 
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager.DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion.assertMarkedAsDeleted;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.TestOrderedSnapshotDeletion.assertNotMarkedAsDeleted;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Test {@link SnapshotDeletionGc}.
  */
 public class TestOrderedSnapshotDeletionGc {
   private static final int GC_PERIOD = 10;
-
+  private static final int NUM_DATANODES = 0;
   private MiniDFSCluster cluster;
 
   @Before
@@ -56,7 +67,8 @@ public class TestOrderedSnapshotDeletionGc {
     conf.setBoolean(DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED, true);
     conf.setInt(DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS, GC_PERIOD);
 
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES)
+        .build();
     cluster.waitActive();
 
     GenericTestUtils.setLogLevel(SnapshotDeletionGc.LOG, Level.TRACE);
@@ -117,6 +129,38 @@ public class TestOrderedSnapshotDeletionGc {
     Assert.assertFalse(exist(s0path, hdfs));
 
     waitForGc(Arrays.asList(s1path, s2path), hdfs);
+    // total no of edit log records created for delete snapshot will be equal
+    // to sum of no of user deleted snapshots and no of snapshots gc'ed with
+    // snapshotDeletion gc thread
+    doEditLogValidation(cluster, 5);
+  }
+
+  static void doEditLogValidation(MiniDFSCluster cluster,
+                                  int editLogOpCount) throws Exception {
+    final FSNamesystem namesystem = cluster.getNamesystem();
+    Configuration conf = cluster.getNameNode().getConf();
+    FSImage fsimage = namesystem.getFSImage();
+    Storage.StorageDirectory sd = fsimage.getStorage().
+        dirIterator(NNStorage.NameNodeDirType.EDITS).next();
+    cluster.shutdown();
+
+    File editFile = FSImageTestUtil.findLatestEditsLog(sd).getFile();
+    assertTrue("Should exist: " + editFile, editFile.exists());
+    EnumMap<FSEditLogOpCodes, Holder<Integer>> counts;
+    counts = FSImageTestUtil.countEditLogOpTypes(editFile);
+    if (editLogOpCount > 0) {
+      assertEquals(editLogOpCount, (int) counts.get(FSEditLogOpCodes.
+          OP_DELETE_SNAPSHOT).held);
+    }
+    // make sure the gc thread doesn't start for a long time after the restart
+    conf.setInt(DFS_NAMENODE_SNAPSHOT_DELETION_ORDERED_GC_PERIOD_MS,
+        (int)(24 * 60_000L));
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES)
+        .build();
+    cluster.waitActive();
+    // ensure after the edits get replayed , all the snapshots are deleted
+    Assert.assertEquals(0,
+        cluster.getNamesystem().getSnapshotManager().getNumSnapshots());
   }
 
   static boolean exist(Path snapshotRoot, DistributedFileSystem hdfs)
@@ -168,6 +212,9 @@ public class TestOrderedSnapshotDeletionGc {
     }
 
     waitForGc(snapshotPaths, hdfs);
+    // don't do edit log count validation here as gc snapshot
+    // deletion count will be random here
+    doEditLogValidation(cluster, -1);
   }
 
   static void createSnapshots(Path snapshottableDir, int numSnapshots,