فهرست منبع

HDFS-4205. fsck fails with symlinks. Contributed by Jason Lowe

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1495648 13f79535-47bb-0310-9956-ffa450edef68
Jason Darrell Lowe 12 سال پیش
والد
کامیت
1087be1df4

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

@@ -611,6 +611,8 @@ Release 2.1.0-beta - UNRELEASED
     HDFS-4917. Start-dfs.sh cannot pass the parameters correctly.
     (Fengdong Yu via suresh)
 
+    HDFS-4205. fsck fails with symlinks. (jlowe)
+
   BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
 
     HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.
@@ -3194,6 +3196,8 @@ Release 0.23.9 - UNRELEASED
     HDFS-4878. On Remove Block, block is not removed from neededReplications
     queue. (Tao Luo via shv)
 
+    HDFS-4205. fsck fails with symlinks. (jlowe)
+
 Release 0.23.8 - 2013-06-05
 
   INCOMPATIBLE CHANGES

+ 9 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -291,6 +291,13 @@ public class NamenodeFsck {
       } while (thisListing.hasMore());
       return;
     }
+    if (file.isSymlink()) {
+      if (showFiles) {
+        out.println(path + " <symlink>");
+      }
+      res.totalSymlinks++;
+      return;
+    }
     long fileLen = file.getLen();
     // Get block locations without updating the file access time 
     // and without block access tokens
@@ -671,6 +678,7 @@ public class NamenodeFsck {
     long totalFiles = 0L;
     long totalOpenFiles = 0L;
     long totalDirs = 0L;
+    long totalSymlinks = 0L;
     long totalSize = 0L;
     long totalOpenFilesSize = 0L;
     long totalReplicas = 0L;
@@ -713,6 +721,7 @@ public class NamenodeFsck {
       }
       res.append("\n Total dirs:\t").append(totalDirs).append(
           "\n Total files:\t").append(totalFiles);
+      res.append("\n Total symlinks:\t\t").append(totalSymlinks);
       if (totalOpenFiles != 0) {
         res.append(" (Files currently being written: ").append(totalOpenFiles)
             .append(")");

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

@@ -49,6 +49,7 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
@@ -874,4 +875,44 @@ public class TestFsck {
       }
     }
   }
+
+  /** Test fsck with symlinks in the filesystem */
+  @Test
+  public void testFsckSymlink() throws Exception {
+    final DFSTestUtil util = new DFSTestUtil.Builder().
+        setName(getClass().getSimpleName()).setNumFiles(1).build();
+    final Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
+
+    MiniDFSCluster cluster = null;
+    FileSystem fs = null;
+    try {
+      final long precision = 1L;
+      conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
+      conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(4).build();
+      fs = cluster.getFileSystem();
+      final String fileName = "/srcdat";
+      util.createFiles(fs, fileName);
+      final FileContext fc = FileContext.getFileContext(
+          cluster.getConfiguration(0));
+      final Path file = new Path(fileName);
+      final Path symlink = new Path("/srcdat-symlink");
+      fc.createSymlink(file, symlink, false);
+      util.waitReplication(fs, fileName, (short)3);
+      long aTime = fc.getFileStatus(symlink).getAccessTime();
+      Thread.sleep(precision);
+      setupAuditLogs();
+      String outStr = runFsck(conf, 0, true, "/");
+      verifyAuditLogs();
+      assertEquals(aTime, fc.getFileStatus(symlink).getAccessTime());
+      assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS));
+      assertTrue(outStr.contains("Total symlinks:\t\t1\n"));
+      System.out.println(outStr);
+      util.cleanup(fs, fileName);
+    } finally {
+      if (fs != null) {try{fs.close();} catch(Exception e){}}
+      if (cluster != null) { cluster.shutdown(); }
+    }
+  }
 }