Forráskód Böngészése

HDFS-17010. Add a subtree test to TestSnapshotDiffReport. (#5656)

Tsz-Wo Nicholas Sze 2 éve
szülő
commit
350dcaf616

+ 36 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapRootDescendantDiff.java

@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -19,10 +19,13 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 
 import static org.junit.Assert.fail;
 import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
@@ -32,7 +35,19 @@ import org.junit.Test;
 /**
  * Test snapshot diff report for the snapshot root descendant directory.
  */
-public class TestSnapRootDescendantDiff extends TestSnapshotDiffReport {
+public class TestSnapRootDescendantDiff {
+  {
+    SnapshotTestHelper.disableLogs();
+  }
+
+  private final Path dir = new Path("/" + getClass().getSimpleName());
+  private final Path sub1 = new Path(dir, "sub1");
+
+  protected Configuration conf;
+  protected MiniDFSCluster cluster;
+  protected DistributedFileSystem hdfs;
+  private final Map<Path, Integer> snapshotNumberMap = new HashMap<>();
+
   @Before
   public void setUp() throws Exception {
     conf = new Configuration();
@@ -59,6 +74,25 @@ public class TestSnapRootDescendantDiff extends TestSnapshotDiffReport {
     }
   }
 
+  private Path getSnapRootDir() {
+    return sub1;
+  }
+
+  private String genSnapshotName(Path snapshotDir) {
+    int sNum = -1;
+    if (snapshotNumberMap.containsKey(snapshotDir)) {
+      sNum = snapshotNumberMap.get(snapshotDir);
+    }
+    snapshotNumberMap.put(snapshotDir, ++sNum);
+    return "s" + sNum;
+  }
+
+  void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
+      throws Exception {
+    TestSnapshotDiffReport.modifyAndCreateSnapshot(
+        modifyDir, snapshotDirs, hdfs, this::genSnapshotName);
+  }
+
   @Test
   public void testNonSnapRootDiffReport() throws Exception {
     Path subsub1 = new Path(getSnapRootDir(), "subsub1");

+ 67 - 21
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDiffReport.java

@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -30,6 +30,7 @@ import java.util.HashMap;
 import java.util.Random;
 import java.util.List;
 import java.util.ArrayList;
+import java.util.function.Function;
 
 import org.apache.commons.collections.list.TreeList;
 import org.apache.hadoop.conf.Configuration;
@@ -50,6 +51,8 @@ import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReportListing;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hdfs.protocol.SnapshotStatus;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
@@ -58,7 +61,6 @@ import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.Assert;
-import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Test;
 import org.slf4j.Logger;
@@ -70,6 +72,10 @@ import org.slf4j.LoggerFactory;
 public class TestSnapshotDiffReport {
   private static final Logger LOG =
       LoggerFactory.getLogger(TestSnapshotDiffReport.class);
+
+  {
+    SnapshotTestHelper.disableLogs();
+  }
   private static final long SEED = 0;
   private static final short REPLICATION = 3;
   private static final short REPLICATION_1 = 2;
@@ -80,9 +86,9 @@ public class TestSnapshotDiffReport {
   private final Path dir = new Path("/TestSnapshot");
   private final Path sub1 = new Path(dir, "sub1");
   
-  protected Configuration conf;
-  protected MiniDFSCluster cluster;
-  protected DistributedFileSystem hdfs;
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem hdfs;
   private final HashMap<Path, Integer> snapshotNumberMap = new HashMap<Path, Integer>();
 
   @Before
@@ -112,10 +118,6 @@ public class TestSnapshotDiffReport {
     }
   }
 
-  protected Path getSnapRootDir() {
-    return sub1;
-  }
-
   private String genSnapshotName(Path snapshotDir) {
     int sNum = -1;
     if (snapshotNumberMap.containsKey(snapshotDir)) {
@@ -125,11 +127,16 @@ public class TestSnapshotDiffReport {
     return "s" + sNum;
   }
 
+  void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
+      throws Exception {
+    modifyAndCreateSnapshot(modifyDir, snapshotDirs, hdfs, this::genSnapshotName);
+  }
   /**
    * Create/modify/delete files under a given directory, also create snapshots
    * of directories.
    */
-  protected void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs)
+  static void modifyAndCreateSnapshot(Path modifyDir, Path[] snapshotDirs,
+      DistributedFileSystem hdfs, Function<Path, String> getSnapshotName)
       throws Exception {
     Path file10 = new Path(modifyDir, "file10");
     Path file11 = new Path(modifyDir, "file11");
@@ -147,7 +154,7 @@ public class TestSnapshotDiffReport {
     // create snapshot
     for (Path snapshotDir : snapshotDirs) {
       hdfs.allowSnapshot(snapshotDir);
-      hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
+      hdfs.createSnapshot(snapshotDir, getSnapshotName.apply(snapshotDir));
     }
 
     // delete file11
@@ -165,7 +172,7 @@ public class TestSnapshotDiffReport {
 
     // create snapshot
     for (Path snapshotDir : snapshotDirs) {
-      hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
+      hdfs.createSnapshot(snapshotDir, getSnapshotName.apply(snapshotDir));
     }
 
     // create file11 again
@@ -183,7 +190,7 @@ public class TestSnapshotDiffReport {
 
     // create snapshot
     for (Path snapshotDir : snapshotDirs) {
-      hdfs.createSnapshot(snapshotDir, genSnapshotName(snapshotDir));
+      hdfs.createSnapshot(snapshotDir, getSnapshotName.apply(snapshotDir));
     }
     // modify file10
     hdfs.setReplication(file10, (short) (REPLICATION + 1));
@@ -317,10 +324,6 @@ public class TestSnapshotDiffReport {
 
   @Test(timeout = 60000)
   public void testSnapRootDescendantDiffReport() throws Exception {
-    Assume.assumeTrue(conf.getBoolean(
-        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT,
-        DFSConfigKeys.
-            DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT));
     Path subSub = new Path(sub1, "subsub1");
     Path subSubSub = new Path(subSub, "subsubsub1");
     Path nonSnapDir = new Path(dir, "non_snap");
@@ -579,10 +582,6 @@ public class TestSnapshotDiffReport {
 
   @Test
   public void testSnapRootDescendantDiffReportWithRename() throws Exception {
-    Assume.assumeTrue(conf.getBoolean(
-        DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT,
-        DFSConfigKeys.
-            DFS_NAMENODE_SNAPSHOT_DIFF_ALLOW_SNAP_ROOT_DESCENDANT_DEFAULT));
     Path subSub = new Path(sub1, "subsub1");
     Path subSubSub = new Path(subSub, "subsubsub1");
     Path nonSnapDir = new Path(dir, "non_snap");
@@ -1603,4 +1602,51 @@ public class TestSnapshotDiffReport {
           + "supported for snapshotDiffReport between two snapshots"));
     }
   }
+
+  @Test
+  public void testSubtrees() throws Exception {
+    final Path root = new Path("/");
+    final Path foo = new Path(root, "foo");
+    final Path bar = new Path(foo, "bar");
+    hdfs.mkdirs(bar);
+    modifyAndCreateSnapshot(bar, new Path[]{root});
+
+    final SnapshottableDirectoryStatus[] snapshottables
+        = hdfs.getSnapshottableDirListing();
+    Assert.assertEquals(1, snapshottables.length);
+    Assert.assertEquals(3, snapshottables[0].getSnapshotNumber());
+
+    final SnapshotStatus[] statuses = hdfs.getSnapshotListing(root);
+    Assert.assertEquals(3, statuses.length);
+    for (int i = 0; i < statuses.length; i++) {
+      final SnapshotStatus s = statuses[i];
+      LOG.info("Snapshot #{}: {}", s.getSnapshotID(), s.getFullPath());
+      Assert.assertEquals(i, s.getSnapshotID());
+    }
+
+    for (int i = 0; i <= 2; i++) {
+      for (int j = 0; j <= 2; j++) {
+        assertDiff(root, foo, bar, "s" + i, "s" + j);
+      }
+    }
+  }
+
+  private void assertDiff(Path root, Path foo, Path bar,
+      String from, String to) throws Exception {
+    final String barDiff = diff(bar, from, to);
+    final String fooDiff = diff(foo, from, to);
+    Assert.assertEquals(barDiff, fooDiff.replace("/bar", ""));
+
+    final String rootDiff = diff(root, from, to);
+    Assert.assertEquals(fooDiff, rootDiff.replace("/foo", ""));
+    Assert.assertEquals(barDiff, rootDiff.replace("/foo/bar", ""));
+  }
+
+  private String diff(Path path, String from, String to) throws Exception {
+    final SnapshotDiffReport diff = hdfs.getSnapshotDiffReport(path, from, to);
+    LOG.info("DIFF {} from {} to {}", path, from, to);
+    LOG.info("{}", diff);
+    final String report = diff.toString();
+    return report.substring(report.indexOf(":") + 1);
+  }
 }