浏览代码

HDFS-7611. deleteSnapshot and delete of a file can leave orphaned blocks in the blocksMap on NameNode restart. Contributed by Jing Zhao and Byron Wong.

Jing Zhao 10 年之前
父节点
当前提交
d244574d03

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

@@ -807,6 +807,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7677. DistributedFileSystem#truncate should resolve symlinks. (yliu)
 
+    HDFS-7611. deleteSnapshot and delete of a file can leave orphaned blocks
+    in the blocksMap on NameNode restart. (jing9 and Byron Wong)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 8 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java

@@ -223,20 +223,25 @@ class FSDirDeleteOp {
     // set the parent's modification time
     final INodeDirectory parent = targetNode.getParent();
     parent.updateModificationTime(mtime, latestSnapshot);
+
+    fsd.updateCountForDelete(targetNode, iip);
     if (removed == 0) {
       return 0;
     }
 
-    // collect block
+    // collect block and update quota
     if (!targetNode.isInLatestSnapshot(latestSnapshot)) {
       targetNode.destroyAndCollectBlocks(collectedBlocks, removedINodes);
     } else {
       Quota.Counts counts = targetNode.cleanSubtree(CURRENT_STATE_ID,
           latestSnapshot, collectedBlocks, removedINodes, true);
-      parent.addSpaceConsumed(-counts.get(Quota.NAMESPACE),
-          -counts.get(Quota.DISKSPACE), true);
       removed = counts.get(Quota.NAMESPACE);
+      // TODO: quota verification may fail the deletion here. We should not
+      // count the snapshot diff into quota usage in the future.
+      fsd.updateCount(iip, -counts.get(Quota.NAMESPACE),
+          -counts.get(Quota.DISKSPACE), true);
     }
+
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* FSDirectory.unprotectedDelete: "
           + iip.getPath() + " is removed");

+ 9 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java

@@ -625,9 +625,12 @@ class FSDirRenameOp {
         NameNode.stateChangeLog.warn("DIR* FSDirRenameOp.unprotectedRenameTo:" +
             error);
         throw new IOException(error);
+      } else {
+        // update the quota count if necessary
+        fsd.updateCountForDelete(srcChild, srcIIP);
+        srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1, null);
+        return removedNum;
       }
-      srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1, null);
-      return removedNum;
     }
 
     boolean removeSrc4OldRename() throws IOException {
@@ -638,6 +641,8 @@ class FSDirRenameOp {
             " can not be removed");
         return false;
       } else {
+        // update the quota count if necessary
+        fsd.updateCountForDelete(srcChild, srcIIP);
         srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1, null);
         return true;
       }
@@ -647,6 +652,8 @@ class FSDirRenameOp {
       long removedNum = fsd.removeLastINode(dstIIP);
       if (removedNum != -1) {
         oldDstChild = dstIIP.getLastINode();
+        // update the quota count if necessary
+        fsd.updateCountForDelete(oldDstChild, dstIIP);
         dstIIP = INodesInPath.replace(dstIIP, dstIIP.length() - 1, null);
       }
       return removedNum;

+ 21 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -601,7 +601,22 @@ public class FSDirectory implements Closeable {
       writeUnlock();
     }
   }
-  
+
+  /**
+   * Update the quota usage after deletion. The quota update is only necessary
+   * when image/edits have been loaded and the file/dir to be deleted is not
+   * contained in snapshots.
+   */
+  void updateCountForDelete(final INode inode, final INodesInPath iip)
+      throws QuotaExceededException {
+    if (getFSNamesystem().isImageLoaded() &&
+        !inode.isInLatestSnapshot(iip.getLatestSnapshotId())) {
+      Quota.Counts counts = inode.computeQuotaUsage();
+      updateCount(iip, -counts.get(Quota.NAMESPACE),
+          -counts.get(Quota.DISKSPACE), false);
+    }
+  }
+
   void updateCount(INodesInPath iip, long nsDelta, long dsDelta,
       boolean checkQuota) throws QuotaExceededException {
     updateCount(iip, iip.length() - 1, nsDelta, dsDelta, checkQuota);
@@ -904,11 +919,12 @@ public class FSDirectory implements Closeable {
 
   /**
    * Remove the last inode in the path from the namespace.
-   * Count of each ancestor with quota is also updated.
+   * Note: the caller needs to update the ancestors' quota count.
+   *
    * @return -1 for failing to remove;
    *          0 for removing a reference whose referred inode has other 
    *            reference nodes;
-   *         >0 otherwise. 
+   *          1 otherwise.
    */
   long removeLastINode(final INodesInPath iip) throws QuotaExceededException {
     final int latestSnapshot = iip.getLatestSnapshotId();
@@ -917,19 +933,9 @@ public class FSDirectory implements Closeable {
     if (!parent.removeChild(last, latestSnapshot)) {
       return -1;
     }
-    
-    if (!last.isInLatestSnapshot(latestSnapshot)) {
-      final Quota.Counts counts = last.computeQuotaUsage();
-      updateCountNoQuotaCheck(iip, iip.length() - 1,
-          -counts.get(Quota.NAMESPACE), -counts.get(Quota.DISKSPACE));
 
-      if (INodeReference.tryRemoveReference(last) > 0) {
-        return 0;
-      } else {
-        return counts.get(Quota.NAMESPACE);
-      }
-    }
-    return 1;
+    return (!last.isInLatestSnapshot(latestSnapshot)
+        && INodeReference.tryRemoveReference(last) > 0) ? 0 : 1;
   }
 
   static String normalizePath(String src) {

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -1191,7 +1191,9 @@ public class MiniDFSCluster {
         } catch (InterruptedException e) {
         }
         if (++i > 10) {
-          throw new IOException("Timed out waiting for Mini HDFS Cluster to start");
+          final String msg = "Timed out waiting for Mini HDFS Cluster to start";
+          LOG.error(msg);
+          throw new IOException(msg);
         }
       }
     }

+ 27 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java

@@ -1122,4 +1122,31 @@ public class TestSnapshotDeletion {
     // wait till the cluster becomes active
     cluster.waitClusterUp();
   }
+
+  @Test
+  public void testCorrectNumberOfBlocksAfterRestart() throws IOException {
+    final Path foo = new Path("/foo");
+    final Path bar = new Path(foo, "bar");
+    final Path file = new Path(foo, "file");
+    final String snapshotName = "ss0";
+
+    DFSTestUtil.createFile(hdfs, file, BLOCKSIZE, REPLICATION, seed);
+    hdfs.mkdirs(bar);
+    hdfs.setQuota(foo, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
+    hdfs.setQuota(bar, Long.MAX_VALUE - 1, Long.MAX_VALUE - 1);
+    hdfs.allowSnapshot(foo);
+
+    hdfs.createSnapshot(foo, snapshotName);
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+    hdfs.saveNamespace();
+
+    hdfs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+    hdfs.deleteSnapshot(foo, snapshotName);
+    hdfs.delete(bar, true);
+    hdfs.delete(foo, true);
+
+    long numberOfBlocks = cluster.getNamesystem().getBlocksTotal();
+    cluster.restartNameNode(0);
+    assertEquals(numberOfBlocks, cluster.getNamesystem().getBlocksTotal());
+  }
 }