浏览代码

HDFS-16316.Improve DirectoryScanner: add regular file check related block. (#3861)

jianghuazhu 3 年之前
父节点
当前提交
589695c6a9

+ 21 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java

@@ -38,6 +38,7 @@ import java.nio.charset.StandardCharsets;
 import java.nio.file.AccessDeniedException;
 import java.nio.file.FileSystems;
 import java.nio.file.Files;
+import java.nio.file.LinkOption;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
@@ -533,6 +534,26 @@ public class FileUtil {
     return dst;
   }
 
+  public static boolean isRegularFile(File file) {
+    return isRegularFile(file, true);
+  }
+
+  /**
+   * Check if the file is regular.
+   * @param file The file being checked.
+   * @param allowLinks Whether to allow matching links.
+   * @return Returns the result of checking whether the file is a regular file.
+   */
+  public static boolean isRegularFile(File file, boolean allowLinks) {
+    if (file != null) {
+      if (allowLinks) {
+        return Files.isRegularFile(file.toPath());
+      }
+      return Files.isRegularFile(file.toPath(), LinkOption.NOFOLLOW_LINKS);
+    }
+    return true;
+  }
+
   /**
    * Convert a os-native filename to a path that works for the shell.
    * @param filename The filename to convert

+ 17 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileUtil.java

@@ -1466,6 +1466,23 @@ public class TestFileUtil {
     Assert.assertEquals(file.getAbsolutePath(), result);
   }
 
+  @Test
+  public void testRegularFile() throws IOException {
+    byte[] data = "testRegularData".getBytes();
+    File tmpFile = new File(del, "reg1");
+
+    // write some data to the file
+    FileOutputStream os = new FileOutputStream(tmpFile);
+    os.write(data);
+    os.close();
+    assertTrue(FileUtil.isRegularFile(tmpFile));
+
+    // create a symlink to file
+    File link = new File(del, "reg2");
+    FileUtil.symLink(tmpFile.toString(), link.toString());
+    assertFalse(FileUtil.isRegularFile(link, false));
+  }
+
   /**
    * This test validates the correctness of {@link FileUtil#readLink(File)} when
    * it gets a file in input.

+ 23 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -41,6 +41,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import org.apache.commons.collections.CollectionUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
@@ -540,21 +541,30 @@ public class DirectoryScanner implements Runnable {
           m++;
           continue;
         }
-        // Block file and/or metadata file exists on the disk
-        // Block exists in memory
-        if (info.getBlockFile() == null) {
-          // Block metadata file exits and block file is missing
-          addDifference(diffRecord, statsRecord, info);
-        } else if (info.getGenStamp() != memBlock.getGenerationStamp()
-            || info.getBlockLength() != memBlock.getNumBytes()) {
-          // Block metadata file is missing or has wrong generation stamp,
-          // or block file length is different than expected
+
+        // Block and meta must be regular file
+        boolean isRegular = FileUtil.isRegularFile(info.getBlockFile(), false) &&
+                FileUtil.isRegularFile(info.getMetaFile(), false);
+        if (!isRegular) {
           statsRecord.mismatchBlocks++;
           addDifference(diffRecord, statsRecord, info);
-        } else if (memBlock.compareWith(info) != 0) {
-          // volumeMap record and on-disk files do not match.
-          statsRecord.duplicateBlocks++;
-          addDifference(diffRecord, statsRecord, info);
+        } else {
+          // Block file and/or metadata file exists on the disk
+          // Block exists in memory
+          if (info.getBlockFile() == null) {
+            // Block metadata file exits and block file is missing
+            addDifference(diffRecord, statsRecord, info);
+          } else if (info.getGenStamp() != memBlock.getGenerationStamp()
+                  || info.getBlockLength() != memBlock.getNumBytes()) {
+            // Block metadata file is missing or has wrong generation stamp,
+            // or block file length is different than expected
+            statsRecord.mismatchBlocks++;
+            addDifference(diffRecord, statsRecord, info);
+          } else if (memBlock.compareWith(info) != 0) {
+            // volumeMap record and on-disk files do not match.
+            statsRecord.duplicateBlocks++;
+            addDifference(diffRecord, statsRecord, info);
+          }
         }
         d++;
 

+ 7 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java

@@ -49,6 +49,7 @@ import javax.management.NotCompliantMBeanException;
 import javax.management.ObjectName;
 import javax.management.StandardMBean;
 
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.classification.VisibleForTesting;
 
@@ -2645,6 +2646,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
           Block.getGenerationStamp(diskMetaFile.getName()) :
           HdfsConstants.GRANDFATHER_GENERATION_STAMP;
 
+      final boolean isRegular = FileUtil.isRegularFile(diskMetaFile, false) &&
+          FileUtil.isRegularFile(diskFile, false);
+
       if (vol.getStorageType() == StorageType.PROVIDED) {
         if (memBlockInfo == null) {
           // replica exists on provided store but not in memory
@@ -2812,6 +2816,9 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
             + memBlockInfo.getNumBytes() + " to "
             + memBlockInfo.getBlockDataLength());
         memBlockInfo.setNumBytes(memBlockInfo.getBlockDataLength());
+      } else if (!isRegular) {
+        corruptBlock = new Block(memBlockInfo);
+        LOG.warn("Block:{} is not a regular file.", corruptBlock.getBlockId());
       }
     } finally {
       if (dataNodeMetrics != null) {

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -49,6 +49,7 @@ import org.apache.commons.io.FileUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSClient;
@@ -58,6 +59,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -71,6 +73,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.LazyPersistTestCase;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.AutoCloseableLock;
@@ -507,6 +510,53 @@ public class TestDirectoryScanner {
     }
   }
 
+  @Test(timeout = 600000)
+  public void testRegularBlock() throws Exception {
+    Configuration conf = getConfiguration();
+    cluster = new MiniDFSCluster.Builder(conf).build();
+    try {
+      cluster.waitActive();
+      bpid = cluster.getNamesystem().getBlockPoolId();
+      fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
+      client = cluster.getFileSystem().getClient();
+      conf.setInt(DFSConfigKeys.DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY, 1);
+      // log trace
+      GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer.
+          captureLogs(NameNode.stateChangeLog);
+      // Add files with 5 blocks
+      createFile(GenericTestUtils.getMethodName(), BLOCK_LENGTH * 5, false);
+
+      List<ReplicaInfo> infos = new ArrayList<>(FsDatasetTestUtil.getReplicas(fds, bpid));
+      ReplicaInfo lastReplica = infos.get(infos.size() - 1);
+      ReplicaInfo penultimateReplica = infos.get(infos.size() - 2);
+
+      String blockParent = new File(lastReplica.getBlockURI().getPath()).getParent();
+      File lastBlockFile = new File(blockParent, getBlockFile(lastReplica.getBlockId()));
+      File penultimateBlockFile = new File(blockParent,
+          getBlockFile(penultimateReplica.getBlockId()));
+      FileUtil.symLink(lastBlockFile.toString(), penultimateBlockFile.toString());
+      ExtendedBlock block = new ExtendedBlock(bpid, penultimateReplica.getBlockId());
+
+      scanner = new DirectoryScanner(fds, conf);
+      scanner.setRetainDiffs(true);
+      scanner.reconcile();
+      DirectoryScanner.Stats stats = scanner.stats.get(bpid);
+      assertNotNull(stats);
+      assertEquals(1, stats.mismatchBlocks);
+
+      // check nn log
+      String msg = "*DIR* reportBadBlocks for block: " + bpid + ":" +
+          getBlockFile(block.getBlockId());
+      assertTrue(logCapturer.getOutput().contains(msg));
+    } finally {
+      if (scanner != null) {
+        scanner.shutdown();
+        scanner = null;
+      }
+      cluster.shutdown();
+    }
+  }
+
   @Test(timeout = 600000)
   public void testDirectoryScanner() throws Exception {
     // Run the test with and without parallel scanning