Explorar o código

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

Kihwal Lee %!s(int64=10) %!d(string=hai) anos
pai
achega
c428d303f6

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

@@ -548,6 +548,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-6651. Deletion failure can leak inodes permanently.
     (Jing Zhao via wheat9)
 
+    HDFS-7707. Edit log corruption due to delayed block removal again.
+    (Yongjun Zhang via kihwal)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

@@ -231,6 +231,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;
@@ -5999,13 +6000,22 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     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();