Browse Source

HDFS-7707. Edit log corruption due to delayed block removal again. Contributed by Yongjun Zhang
(cherry picked from commit 843806d03ab1a24f191782f42eb817505228eb9f)

(cherry picked from commit c428d303f67bef3a7df12153947c6b0199a0938b)
(cherry picked from commit 21d8b22feef2fc9f6793d9c157391b142368e588)

Kihwal Lee 10 years ago
parent
commit
a1c3e6a058

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

@@ -67,6 +67,9 @@ Release 2.6.1 - UNRELEASED
     HDFS-7575. Upgrade should generate a unique storage ID for each
     volume. (Arpit Agarwal)
 
+    HDFS-7707. Edit log corruption due to delayed block removal again.
+    (Yongjun Zhang via kihwal)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

+ 12 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -238,6 +238,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.DirectoryWithSnapshotFeature;
 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;
@@ -6864,13 +6865,22 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     INode tmpChild = file;
     INodeDirectory tmpParent = file.getParent();
     while (true) {
-      if (tmpParent == null ||
-          tmpParent.searchChildren(tmpChild.getLocalNameBytes()) < 0) {
+      if (tmpParent == null) {
         return true;
       }
+
+      INode childINode = tmpParent.getChild(tmpChild.getLocalNameBytes(),
+          Snapshot.CURRENT_STATE_ID);
+      if (childINode == null || !childINode.equals(tmpChild)) {
+        // a newly created INode with the same name as an already deleted one
+        // would be a different INode than the deleted one
+        return true;
+      }
+
       if (tmpParent.isRoot()) {
         break;
       }
+
       tmpChild = tmpParent;
       tmpParent = tmpParent.getParent();
     }

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java

@@ -54,7 +54,9 @@ public class TestCommitBlockSynchronization {
     // set file's parent as root and put the file to inodeMap, so
     // FSNamesystem's isFileDeleted() method will return false on this file
     if (file.getParent() == null) {
-      INodeDirectory parent = mock(INodeDirectory.class);
+      INodeDirectory mparent = mock(INodeDirectory.class);
+      INodeDirectory parent = new INodeDirectory(mparent.getId(), new byte[0],
+          mparent.getPermissionStatus(), mparent.getAccessTime());
       parent.setLocalName(new byte[0]);
       parent.addChild(file);
       file.setParent(parent);

+ 25 - 7
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeleteRace.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.FileNotFoundException;
+import java.util.AbstractMap;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -226,10 +228,19 @@ public class TestDeleteRace {
   private void testDeleteAndCommitBlockSynchronizationRace(boolean hasSnapshot)
       throws Exception {
     LOG.info("Start testing, hasSnapshot: " + hasSnapshot);
-    final String testPaths[] = {
-        "/test-file",
-        "/testdir/testdir1/test-file"
-    };
+    ArrayList<AbstractMap.SimpleImmutableEntry<String, Boolean>> testList =
+        new ArrayList<AbstractMap.SimpleImmutableEntry<String, Boolean>> ();
+    testList.add(
+        new AbstractMap.SimpleImmutableEntry<String, Boolean>("/test-file", false));
+    testList.add(     
+        new AbstractMap.SimpleImmutableEntry<String, Boolean>("/test-file1", true));
+    testList.add(
+        new AbstractMap.SimpleImmutableEntry<String, Boolean>(
+            "/testdir/testdir1/test-file", false));
+    testList.add(
+        new AbstractMap.SimpleImmutableEntry<String, Boolean>(
+            "/testdir/testdir1/test-file1", true));
+    
     final Path rootPath = new Path("/");
     final Configuration conf = new Configuration();
     // Disable permissions so that another user can recover the lease.
@@ -247,8 +258,11 @@ public class TestDeleteRace {
 
       DistributedFileSystem fs = cluster.getFileSystem();
       int stId = 0;
-      for (String testPath : testPaths) {
-        LOG.info("test on " + testPath + " snapshot: " + hasSnapshot);
+      for(AbstractMap.SimpleImmutableEntry<String, Boolean> stest : testList) {
+        String testPath = stest.getKey();
+        Boolean mkSameDir = stest.getValue();
+        LOG.info("test on " + testPath + " mkSameDir: " + mkSameDir
+            + " snapshot: " + hasSnapshot);
         Path fPath = new Path(testPath);
         //find grandest non-root parent
         Path grandestNonRootParent = fPath;
@@ -304,7 +318,11 @@ public class TestDeleteRace {
 
         LOG.info("Deleting recursively " + grandestNonRootParent);
         fs.delete(grandestNonRootParent, true);
-
+        if (mkSameDir && !grandestNonRootParent.toString().equals(testPath)) {
+          LOG.info("Recreate dir " + grandestNonRootParent + " testpath: "
+              + testPath);
+          fs.mkdirs(grandestNonRootParent);
+        }
         delayer.proceed();
         LOG.info("Now wait for result");
         delayer.waitForResult();