Browse Source

HDFS-9231. fsck doesn't list correct file path when Bad Replicas/Blocks are in a snapshot. (Xiao Chen via Yongjun Zhang)

(cherry picked from commit 97913f430cbe3f82ac866ae6ab8f42754102f6c0)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
Yongjun Zhang 9 years ago
parent
commit
6fed564d2c

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -1335,6 +1335,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-9268. fuse_dfs chown crashes when uid is passed as -1 (cmccabe)
 
+    HDFS-9231. fsck doesn't list correct file path when Bad Replicas/Blocks
+    are in a snapshot. (Xiao Chen via Yongjun Zhang)
+
 Release 2.7.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 38 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSnapshotOp.java

@@ -29,9 +29,13 @@ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
+import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.util.ChunkedArrayList;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.ListIterator;
 import java.util.List;
 
 class FSDirSnapshotOp {
@@ -159,6 +163,40 @@ class FSDirSnapshotOp {
     return diffs;
   }
 
+  /** Get a collection of full snapshot paths given file and snapshot dir.
+   * @param lsf a list of snapshottable features
+   * @param file full path of the file
+   * @return collection of full paths of snapshot of the file
+   */
+  static Collection<String> getSnapshotFiles(FSDirectory fsd,
+      List<DirectorySnapshottableFeature> lsf,
+      String file) throws IOException {
+    ArrayList<String> snaps = new ArrayList<String>();
+    ListIterator<DirectorySnapshottableFeature> sfi = lsf.listIterator();
+    for (DirectorySnapshottableFeature sf : lsf) {
+      // for each snapshottable dir e.g. /dir1, /dir2
+      final ReadOnlyList<Snapshot> lsnap = sf.getSnapshotList();
+      for (Snapshot s : lsnap) {
+        // for each snapshot name under snapshottable dir
+        // e.g. /dir1/.snapshot/s1, /dir1/.snapshot/s2
+        final String dirName = s.getRoot().getRootFullPathName();
+        if (!file.startsWith(dirName)) {
+          // file not in current snapshot root dir, no need to check other snaps
+          break;
+        }
+        String snapname = s.getRoot().getFullPathName();
+        if (dirName.equals(Path.SEPARATOR)) { // handle rootDir
+          snapname += Path.SEPARATOR;
+        }
+        snapname += file.substring(file.indexOf(dirName) + dirName.length());
+        if (fsd.getFSNamesystem().getFileInfo(snapname, true) != null) {
+          snaps.add(snapname);
+        }
+      }
+    }
+    return snaps;
+  }
+
   /**
    * Delete a snapshot of a snapshottable directory
    * @param snapshotRoot The snapshottable directory

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

@@ -195,6 +195,7 @@ import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.protocol.SnapshotException;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
@@ -231,6 +232,7 @@ import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
 import org.apache.hadoop.hdfs.server.namenode.metrics.FSNamesystemMBean;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
+import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotManager;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase;
@@ -6237,6 +6239,79 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return JSON.toString(list);
   }
 
+  @Override  // NameNodeMXBean
+  public long getNumberOfSnapshottableDirs() {
+    return snapshotManager.getNumSnapshottableDirs();
+  }
+
+  /**
+   * Get the list of corrupt blocks and corresponding full file path
+   * including snapshots in given snapshottable directories.
+   * @param path Restrict corrupt files to this portion of namespace.
+   * @param snapshottableDirs Snapshottable directories. Passing in null
+   *                          will only return corrupt blocks in non-snapshots.
+   * @param cookieTab Support for continuation; cookieTab tells where
+   *                  to start from.
+   * @return a list in which each entry describes a corrupt file/block
+   * @throws IOException
+   */
+  List<String> listCorruptFileBlocksWithSnapshot(String path,
+      List<String> snapshottableDirs, String[] cookieTab) throws IOException {
+    final Collection<CorruptFileBlockInfo> corruptFileBlocks =
+        listCorruptFileBlocks(path, cookieTab);
+    List<String> list = new ArrayList<String>();
+
+    // Precalculate snapshottableFeature list
+    List<DirectorySnapshottableFeature> lsf = new ArrayList<>();
+    if (snapshottableDirs != null) {
+      for (String snap : snapshottableDirs) {
+        final INode isnap = getFSDirectory().getINode(snap, false);
+        final DirectorySnapshottableFeature sf =
+            isnap.asDirectory().getDirectorySnapshottableFeature();
+        if (sf == null) {
+          throw new SnapshotException(
+              "Directory is not a snapshottable directory: " + snap);
+        }
+        lsf.add(sf);
+      }
+    }
+
+    for (CorruptFileBlockInfo c : corruptFileBlocks) {
+      if (getFileInfo(c.path, true) != null) {
+        list.add(c.toString());
+      }
+      final Collection<String> snaps = FSDirSnapshotOp
+          .getSnapshotFiles(getFSDirectory(), lsf, c.path);
+      if (snaps != null) {
+        for (String snap : snaps) {
+          // follow the syntax of CorruptFileBlockInfo#toString()
+          list.add(c.block.getBlockName() + "\t" + snap);
+        }
+      }
+    }
+    return list;
+  }
+
+  /**
+   * Get the list of snapshottable directories.
+   * @return The list of all the current snapshottable directories
+   * @see #getSnapshottableDirListing()
+   * @throws IOException
+   */
+  List<String> getSnapshottableDirs() throws IOException {
+    List<String> snapshottableDirs = new ArrayList<String>();
+    final FSPermissionChecker pc = getFSDirectory().getPermissionChecker();
+    final String user = pc.isSuperUser() ? null : pc.getUser();
+    final SnapshottableDirectoryStatus[] snapDirs =
+        snapshotManager.getSnapshottableDirListing(user);
+    if (snapDirs != null) {
+      for (SnapshottableDirectoryStatus sds : snapDirs) {
+        snapshottableDirs.add(sds.getFullPath().toString());
+      }
+    }
+    return snapshottableDirs;
+  }
+
   @Override  //NameNodeMXBean
   public int getDistinctVersionCount() {
     return blockManager.getDatanodeManager().getDatanodesSoftwareVersions()

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeMXBean.java

@@ -161,6 +161,13 @@ public interface NameNodeMXBean {
    */
   public long getNumberOfMissingBlocksWithReplicationFactorOne();
 
+  /**
+   * Gets the total number of snapshottable dirs in the system.
+   *
+   * @return the total number of snapshottable dirs in the system
+   */
+  public long getNumberOfSnapshottableDirs();
+
   /**
    * Gets the number of threads.
    * 

+ 7 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -60,7 +60,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
@@ -338,13 +337,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       namenode.getNamesystem().logFsckEvent(path, remoteAddress);
 
       if (snapshottableDirs != null) {
-        SnapshottableDirectoryStatus[] snapshotDirs = namenode.getRpcServer()
-            .getSnapshottableDirListing();
-        if (snapshotDirs != null) {
-          for (SnapshottableDirectoryStatus dir : snapshotDirs) {
-            snapshottableDirs.add(dir.getFullPath().toString());
-          }
-        }
+        snapshottableDirs = namenode.getNamesystem().getSnapshottableDirs();
       }
 
       final HdfsFileStatus file = namenode.getRpcServer().getFileInfo(path);
@@ -408,9 +401,10 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   }
 
   private void listCorruptFileBlocks() throws IOException {
-    Collection<FSNamesystem.CorruptFileBlockInfo> corruptFiles = namenode.
-      getNamesystem().listCorruptFileBlocks(path, currentCookie);
-    int numCorruptFiles = corruptFiles.size();
+    final List<String> corrputBlocksFiles = namenode.getNamesystem()
+        .listCorruptFileBlocksWithSnapshot(path, snapshottableDirs,
+            currentCookie);
+    int numCorruptFiles = corrputBlocksFiles.size();
     String filler;
     if (numCorruptFiles > 0) {
       filler = Integer.toString(numCorruptFiles);
@@ -420,8 +414,8 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       filler = "no more";
     }
     out.println("Cookie:\t" + currentCookie[0]);
-    for (FSNamesystem.CorruptFileBlockInfo c : corruptFiles) {
-      out.println(c.toString());
+    for (String s : corrputBlocksFiles) {
+      out.println(s);
     }
     out.println("\n\nThe filesystem under path '" + path + "' has " + filler
         + " CORRUPT files");

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/Snapshot.java

@@ -184,6 +184,14 @@ public class Snapshot implements Comparable<byte[]> {
     public String getFullPathName() {
       return getSnapshotPath(getParent().getFullPathName(), getLocalName());
     }
+
+    /**
+     * Get the full path name of the root directory of this snapshot.
+     * @return full path to the root directory of the snapshot
+     */
+    public String getRootFullPathName() {
+      return getParent().getFullPathName();
+    }
   }
 
   /** Snapshot ID. */

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/dfshealth.html

@@ -117,7 +117,7 @@
     {.}<br/>
     {/CorruptFiles}
   </div>
-  <p>Please check the logs or run fsck in order to identify the missing blocks. See the Hadoop FAQ for common causes and potential solutions.</p>
+  <p>Please check the logs or run fsck in order to identify the missing blocks.{@if cond="{NumberOfSnapshottableDirs} > 0"} Please run fsck with -includeSnapshots in order to see detailed reports about snapshots.{/if} See the Hadoop FAQ for common causes and potential solutions.</p>
 </div>
 {/if}
 {/nn}

+ 88 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java

@@ -1646,4 +1646,92 @@ public class TestFsck {
       }
     }
   }
+
+  /**
+   * Test that corrupted snapshot files are listed with full dir.
+   */
+  @Test
+  public void testFsckListCorruptSnapshotFiles() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 1000);
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_INTERVAL_KEY, 1);
+    DistributedFileSystem hdfs = null;
+    final short REPL_FACTOR = 1;
+
+    MiniDFSCluster cluster = null;
+    try {
+      int numFiles = 3;
+      int numSnapshots = 0;
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      hdfs = cluster.getFileSystem();
+      DFSTestUtil util = new DFSTestUtil.Builder().
+          setName("testGetCorruptFiles").setNumFiles(numFiles).setMaxLevels(1).
+          setMaxSize(1024).build();
+
+      util.createFiles(hdfs, "/corruptData", (short) 1);
+      final Path fp = new Path("/corruptData/file");
+      util.createFile(hdfs, fp, 1024, REPL_FACTOR, 1000L);
+      numFiles++;
+      util.waitReplication(hdfs, "/corruptData", (short) 1);
+
+      hdfs.allowSnapshot(new Path("/corruptData"));
+      hdfs.createSnapshot(new Path("/corruptData"), "mySnapShot");
+      numSnapshots = numFiles;
+
+      String outStr =
+          runFsck(conf, 0, false, "/corruptData", "-list-corruptfileblocks");
+      System.out.println("1. good fsck out: " + outStr);
+      assertTrue(outStr.contains("has 0 CORRUPT files"));
+      // delete the blocks
+      final String bpid = cluster.getNamesystem().getBlockPoolId();
+      for (int i=0; i<numFiles; i++) {
+        for (int j=0; j<=1; j++) {
+          File storageDir = cluster.getInstanceStorageDir(i, j);
+          File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, bpid);
+          List<File> metadataFiles = MiniDFSCluster.getAllBlockMetadataFiles(
+              data_dir);
+          if (metadataFiles == null)
+            continue;
+          for (File metadataFile : metadataFiles) {
+            File blockFile = Block.metaToBlockFile(metadataFile);
+            assertTrue("Cannot remove file.", blockFile.delete());
+            assertTrue("Cannot remove file.", metadataFile.delete());
+          }
+        }
+      }
+      // Delete file when it has a snapshot
+      hdfs.delete(fp, false);
+      numFiles--;
+
+      // wait for the namenode to see the corruption
+      final NamenodeProtocols namenode = cluster.getNameNodeRpc();
+      CorruptFileBlocks corruptFileBlocks = namenode
+          .listCorruptFileBlocks("/corruptData", null);
+      int numCorrupt = corruptFileBlocks.getFiles().length;
+      while (numCorrupt == 0) {
+        Thread.sleep(1000);
+        corruptFileBlocks = namenode
+            .listCorruptFileBlocks("/corruptData", null);
+        numCorrupt = corruptFileBlocks.getFiles().length;
+      }
+
+      // with -includeSnapshots all files are reported
+      outStr = runFsck(conf, -1, true, "/corruptData",
+          "-list-corruptfileblocks", "-includeSnapshots");
+      System.out.println("2. bad fsck include snapshot out: " + outStr);
+      assertTrue(outStr
+          .contains("has " + (numFiles + numSnapshots) + " CORRUPT files"));
+      assertTrue(outStr.contains("/.snapshot/"));
+
+      // without -includeSnapshots only non-snapshots are reported
+      outStr =
+          runFsck(conf, -1, true, "/corruptData", "-list-corruptfileblocks");
+      System.out.println("3. bad fsck exclude snapshot out: " + outStr);
+      assertTrue(outStr.contains("has " + numFiles + " CORRUPT files"));
+      assertFalse(outStr.contains("/.snapshot/"));
+    } finally {
+      if (cluster != null) {cluster.shutdown();}
+    }
+  }
 }