浏览代码

HDFS-7903. Cannot recover block after truncate and delete snapshot. Contributed by Plamen Jeliazkov.

Konstantin V Shvachko 10 年之前
父节点
当前提交
f6bdcd938e

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

@@ -833,6 +833,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7926. NameNode implementation of ClientProtocol.truncate(..) is not 
     idempotent (Tsz Wo Nicholas Sze via brandonli)
 
+    HDFS-7903. Cannot recover block after truncate and delete snapshot.
+    (Plamen Jeliazkov via shv)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

+ 16 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java

@@ -20,8 +20,11 @@ package org.apache.hadoop.hdfs.server.namenode.snapshot;
 import java.util.Collections;
 import java.util.List;
 
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
@@ -125,9 +128,19 @@ public class FileDiffList extends
         continue;
       break;
     }
-    // Collect the remaining blocks of the file
-    while(i < removedBlocks.length) {
-      collectedBlocks.addDeleteBlock(removedBlocks[i++]);
+    // Check if last block is part of truncate recovery
+    BlockInfoContiguous lastBlock = file.getLastBlock();
+    Block dontRemoveBlock = null;
+    if(lastBlock != null && lastBlock.getBlockUCState().equals(
+        HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
+      dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)
+          .getTruncateBlock();
+    }
+    // Collect the remaining blocks of the file, ignoring truncate block
+    for(;i < removedBlocks.length; i++) {
+      if(dontRemoveBlock == null || !removedBlocks[i].equals(dontRemoveBlock)) {
+        collectedBlocks.addDeleteBlock(removedBlocks[i]);
+      }
     }
   }
 }

+ 30 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java

@@ -178,6 +178,36 @@ public class TestFileTruncate {
     fs.delete(dir, true);
   }
 
+  /** Truncate the same file multiple times until its size is zero. */
+  @Test
+  public void testSnapshotTruncateThenDeleteSnapshot() throws IOException {
+    Path dir = new Path("/testSnapshotTruncateThenDeleteSnapshot");
+    fs.mkdirs(dir);
+    fs.allowSnapshot(dir);
+    final Path p = new Path(dir, "file");
+    final byte[] data = new byte[BLOCK_SIZE];
+    DFSUtil.getRandom().nextBytes(data);
+    writeContents(data, data.length, p);
+    final String snapshot = "s0";
+    fs.createSnapshot(dir, snapshot);
+    Block lastBlock = getLocatedBlocks(p).getLastLocatedBlock()
+        .getBlock().getLocalBlock();
+    final int newLength = data.length - 1;
+    assert newLength % BLOCK_SIZE != 0 :
+        " newLength must not be multiple of BLOCK_SIZE";
+    final boolean isReady = fs.truncate(p, newLength);
+    LOG.info("newLength=" + newLength + ", isReady=" + isReady);
+    assertEquals("File must be closed for truncating at the block boundary",
+        isReady, newLength % BLOCK_SIZE == 0);
+    fs.deleteSnapshot(dir, snapshot);
+    if (!isReady) {
+      checkBlockRecovery(p);
+    }
+    checkFullFile(p, newLength, data);
+    assertBlockNotPresent(lastBlock);
+    fs.delete(dir, true);
+  }
+
   /**
    * Truncate files and then run other operations such as
    * rename, set replication, set permission, etc.