You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by we...@apache.org on 2019/08/01 22:49:38 UTC

[hadoop] branch branch-3.2 updated: HDFS-14631.The DirectoryScanner doesn't fix the wrongly placed replica. Contributed by Jinglun.

This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a commit to branch branch-3.2
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/branch-3.2 by this push:
     new 5c1f91f  HDFS-14631.The DirectoryScanner doesn't fix the wrongly placed replica. Contributed by Jinglun.
5c1f91f is described below

commit 5c1f91fd9757a5a788afaa2a1d64433cd34df117
Author: Wei-Chiu Chuang <we...@apache.org>
AuthorDate: Thu Aug 1 15:48:36 2019 -0700

    HDFS-14631.The DirectoryScanner doesn't fix the wrongly placed replica. Contributed by Jinglun.
    
    (cherry picked from commit 32607dbd98a7ab70741a2efc98eff548c1e431c1)
---
 .../hadoop/hdfs/server/datanode/LocalReplica.java  | 15 +++--
 .../hdfs/server/datanode/TestDirectoryScanner.java | 75 +++++++++++++++++++++-
 2 files changed, 83 insertions(+), 7 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
index 68126a5..cb53a64 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplica.java
@@ -137,7 +137,7 @@ abstract public class LocalReplica extends ReplicaInfo {
       return;
     }
 
-    ReplicaDirInfo dirInfo = parseBaseDir(dir);
+    ReplicaDirInfo dirInfo = parseBaseDir(dir, getBlockId());
     this.hasSubdirs = dirInfo.hasSubidrs;
 
     synchronized (internedBaseDirs) {
@@ -163,16 +163,21 @@ abstract public class LocalReplica extends ReplicaInfo {
   }
 
   @VisibleForTesting
-  public static ReplicaDirInfo parseBaseDir(File dir) {
-
+  public static ReplicaDirInfo parseBaseDir(File dir, long blockId) {
     File currentDir = dir;
     boolean hasSubdirs = false;
     while (currentDir.getName().startsWith(DataStorage.BLOCK_SUBDIR_PREFIX)) {
       hasSubdirs = true;
       currentDir = currentDir.getParentFile();
     }
-
-    return new ReplicaDirInfo(currentDir.getAbsolutePath(), hasSubdirs);
+    if (hasSubdirs) {
+      // set baseDir to currentDir if it matches id(idToBlockDir).
+      File idToBlockDir = DatanodeUtil.idToBlockDir(currentDir, blockId);
+      if (idToBlockDir.equals(dir)) {
+        return new ReplicaDirInfo(currentDir.getAbsolutePath(), true);
+      }
+    }
+    return new ReplicaDirInfo(dir.getAbsolutePath(), false);
   }
 
   /**
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
index dcb68e0..438119a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDirectoryScanner.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
+import static org.apache.hadoop.hdfs.protocol.Block.BLOCK_FILE_PREFIX;
 import static org.apache.hadoop.util.Shell.getMemlockLimit;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.core.Is.is;
@@ -64,6 +65,7 @@ import org.apache.hadoop.util.AutoCloseableLock;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DirectoryScanner.ReportCompiler;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
@@ -240,11 +242,11 @@ public class TestDirectoryScanner {
   }
 
   private String getBlockFile(long id) {
-    return Block.BLOCK_FILE_PREFIX + id;
+    return BLOCK_FILE_PREFIX + id;
   }
 
   private String getMetaFile(long id) {
-    return Block.BLOCK_FILE_PREFIX + id + "_" + DEFAULT_GEN_STAMP
+    return BLOCK_FILE_PREFIX + id + "_" + DEFAULT_GEN_STAMP
         + Block.METADATA_EXTENSION;
   }
 
@@ -1173,6 +1175,75 @@ public class TestDirectoryScanner {
     }
   }
 
+  private static final String SEP = System.getProperty("file.separator");
+
+  /**
+   * Test parsing LocalReplica. We should be able to find the replica's path
+   * even if the replica's dir doesn't match the idToBlockDir.
+   */
+  @Test(timeout = 3000)
+  public void testLocalReplicaParsing() {
+    String baseDir = GenericTestUtils.getRandomizedTempPath();
+    long blkId = getRandomBlockId();
+    File blockDir = DatanodeUtil.idToBlockDir(new File(baseDir), blkId);
+    String subdir1 = new File(blockDir.getParent()).getName();
+
+    // test parsing dir without ./subdir/subdir
+    LocalReplica.ReplicaDirInfo info =
+        LocalReplica.parseBaseDir(new File(baseDir), blkId);
+    assertEquals(baseDir, info.baseDirPath);
+    assertEquals(false, info.hasSubidrs);
+
+    // test when path doesn't match the idToBLockDir.
+    String pathWithOneSubdir = baseDir + SEP + subdir1;
+    info = LocalReplica.parseBaseDir(new File(pathWithOneSubdir), blkId);
+    assertEquals(pathWithOneSubdir, info.baseDirPath);
+    assertEquals(false, info.hasSubidrs);
+
+    // test when path doesn't match the idToBlockDir.
+    String badPath = baseDir + SEP + subdir1 + SEP + "subdir-not-exist";
+    info = LocalReplica.parseBaseDir(new File(badPath), blkId);
+    assertEquals(badPath, info.baseDirPath);
+    assertEquals(false, info.hasSubidrs);
+
+    // test when path matches the idToBlockDir.
+    info = LocalReplica.parseBaseDir(blockDir, blkId);
+    assertEquals(baseDir, info.baseDirPath);
+    assertEquals(true, info.hasSubidrs);
+  }
+
+  /**
+   * Test whether can LocalReplica.updateWithReplica() correct the wrongly
+   * recorded replica location.
+   */
+  @Test(timeout = 3000)
+  public void testLocalReplicaUpdateWithReplica() throws Exception {
+    String baseDir = GenericTestUtils.getRandomizedTempPath();
+    long blkId = getRandomBlockId();
+    File blockDir = DatanodeUtil.idToBlockDir(new File(baseDir), blkId);
+    String subdir2 = blockDir.getName();
+    String subdir1 = new File(blockDir.getParent()).getName();
+    String diskSub = subdir2.equals("subdir0") ? "subdir1" : "subdir0";
+
+    // the block file on disk
+    File diskBlockDir = new File(baseDir + SEP + subdir1 + SEP + diskSub);
+    File realBlkFile = new File(diskBlockDir, BLOCK_FILE_PREFIX + blkId);
+    // the block file in mem
+    File memBlockDir = blockDir;
+    LocalReplica localReplica = (LocalReplica) new ReplicaBuilder(
+        HdfsServerConstants.ReplicaState.FINALIZED)
+        .setDirectoryToUse(memBlockDir).setBlockId(blkId).build();
+
+    // DirectoryScanner find the inconsistent file and try to make it right
+    StorageLocation sl = StorageLocation.parse(realBlkFile.toString());
+    localReplica.updateWithReplica(sl);
+    assertEquals(realBlkFile, localReplica.getBlockFile());
+  }
+
+  public long getRandomBlockId() {
+    return Math.abs(new Random().nextLong());
+  }
+
   private void writeFile(FileSystem fs, int numFiles) throws IOException {
     final String fileName = "/" + GenericTestUtils.getMethodName();
     final Path filePath = new Path(fileName);


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org