فهرست منبع

HDFS-12319. DirectoryScanner will throw IllegalStateException when Multiple BP's are present. Contributed by Brahma Reddy Battula.

(cherry picked from commit 2cae387402392f9dadadc95b9bc1677d93515758)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
Arpit Agarwal 7 سال پیش
والد
کامیت
bd79bfc3a0

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

@@ -1,5 +1,20 @@
 Hadoop HDFS Change Log
 
+Release 2.7.8 - UNRELEASED
+
+  INCOMPATIBLE CHANGES
+
+  NEW FEATURES
+
+  IMPROVEMENTS
+
+  OPTIMIZATIONS
+
+  BUG FIXES
+
+    HDFS-12319. DirectoryScanner will throw IllegalStateException
+    when Multiple BP's are present. Contributed by Brahma Reddy Battula.
+
 Release 2.7.7 - 2018-07-18
 
   INCOMPATIBLE CHANGES

+ 2 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DirectoryScanner.java

@@ -818,13 +818,12 @@ public class DirectoryScanner implements Runnable {
     public ScanInfoPerBlockPool call() throws IOException {
       String[] bpList = volume.getBlockPoolList();
       ScanInfoPerBlockPool result = new ScanInfoPerBlockPool(bpList.length);
+      perfTimer.start();
+      throttleTimer.start();
       for (String bpid : bpList) {
         LinkedList<ScanInfo> report = new LinkedList<>();
         File bpFinalizedDir = volume.getFinalizedDir(bpid);
 
-        perfTimer.start();
-        throttleTimer.start();
-
         try {
           result.put(bpid,
               compileReport(volume, bpFinalizedDir, bpFinalizedDir, report));

+ 61 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 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.LocatedBlock;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
@@ -277,15 +278,22 @@ public class TestDirectoryScanner {
          missingMemoryBlocks, mismatchBlocks, 0);
   }
 
-    private void scan(long totalBlocks, int diffsize, long missingMetaFile, long missingBlockFile,
-      long missingMemoryBlocks, long mismatchBlocks, long duplicateBlocks) throws IOException {
+  private void scan(long totalBlocks, int diffsize, long missingMetaFile,
+      long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks,
+      long duplicateBlocks) throws IOException {
     scanner.reconcile();
-    
+    verifyStats(totalBlocks, diffsize, missingMetaFile, missingBlockFile,
+        missingMemoryBlocks, mismatchBlocks, duplicateBlocks);
+  }
+
+  private void verifyStats(long totalBlocks, int diffsize, long missingMetaFile,
+      long missingBlockFile, long missingMemoryBlocks, long mismatchBlocks,
+      long duplicateBlocks) {
     assertTrue(scanner.diffs.containsKey(bpid));
     LinkedList<DirectoryScanner.ScanInfo> diff = scanner.diffs.get(bpid);
     assertTrue(scanner.stats.containsKey(bpid));
     DirectoryScanner.Stats stats = scanner.stats.get(bpid);
-    
+
     assertEquals(diffsize, diff.size());
     assertEquals(totalBlocks, stats.totalBlocks);
     assertEquals(missingMetaFile, stats.missingMetaFile);
@@ -904,4 +912,53 @@ public class TestDirectoryScanner {
         new File(TEST_VOLUME.getFinalizedDir(BPID_2).getAbsolutePath(),
             "blk_567__1004.meta"));
   }
+
+  @Test
+  public void testDirectoryScannerInFederatedCluster() throws Exception {
+    //Create Federated cluster with two nameservices and one DN
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(CONF)
+          .nnTopology(MiniDFSNNTopology.simpleHAFederatedTopology(2))
+          .numDataNodes(1).build();
+      cluster.waitActive();
+      cluster.transitionToActive(1);
+      cluster.transitionToActive(3);
+      DataNode dataNode = cluster.getDataNodes().get(0);
+      fds = DataNodeTestUtils.getFSDataset(cluster.getDataNodes().get(0));
+      //Create one block in first nameservice
+      FileSystem fs = cluster.getFileSystem(1);
+      int bp1Files = 1;
+      writeFile(fs, bp1Files);
+      //Create two blocks in second nameservice
+      FileSystem fs2 = cluster.getFileSystem(3);
+      int bp2Files = 2;
+      writeFile(fs2, bp2Files);
+      //Call the Directory scanner
+      scanner = new DirectoryScanner(dataNode, fds, CONF);
+      scanner.setRetainDiffs(true);
+      scanner.reconcile();
+      //Check blocks in corresponding BP
+      bpid = cluster.getNamesystem(1).getBlockPoolId();
+      verifyStats(bp1Files, 0, 0, 0, 0, 0, 0);
+      bpid = cluster.getNamesystem(3).getBlockPoolId();
+      verifyStats(bp2Files, 0, 0, 0, 0, 0, 0);
+    } finally {
+      if (scanner != null) {
+        scanner.shutdown();
+        scanner = null;
+      }
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  private void writeFile(FileSystem fs, int numFiles) throws IOException {
+    final String fileName = "/" + GenericTestUtils.getMethodName();
+    final Path filePath = new Path(fileName);
+    for (int i = 0; i < numFiles; i++) {
+      DFSTestUtil.createFile(fs, filePath, 1, (short) 1, 0);
+    }
+  }
 }