浏览代码

HDFS-2773. Reading edit logs from an earlier version should not leave blocks in under-construction state. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1229900 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 年之前
父节点
当前提交
298e867673

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt

@@ -95,3 +95,5 @@ HDFS-2762. Fix TestCheckpoint timing out on HA branch. (Uma Maheswara Rao G via
 HDFS-2724. NN web UI can throw NPE after startup, before standby state is entered. (todd)
 
 HDFS-2753. Fix standby getting stuck in safemode when blocks are written while SBN is down. (Hari Mankude and todd via todd)
+
+HDFS-2773. Reading edit logs from an earlier version should not leave blocks in under-construction state. (todd)

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

@@ -467,7 +467,7 @@ public class FSEditLogLoader {
       BlockInfo oldBlock = oldBlocks[i];
       Block newBlock = addCloseOp.blocks[i];
       
-      boolean isLastBlock = i == oldBlocks.length - 1;
+      boolean isLastBlock = i == addCloseOp.blocks.length - 1;
       if (oldBlock.getBlockId() != newBlock.getBlockId() ||
           (oldBlock.getGenerationStamp() != newBlock.getGenerationStamp() && 
               !(isGenStampUpdate && isLastBlock))) {
@@ -504,7 +504,17 @@ public class FSEditLogLoader {
       // We're adding blocks
       for (int i = oldBlocks.length; i < addCloseOp.blocks.length; i++) {
         Block newBlock = addCloseOp.blocks[i];
-        BlockInfo newBI = new BlockInfoUnderConstruction(newBlock, file.getReplication());
+        BlockInfo newBI;
+        if (addCloseOp.opCode == FSEditLogOpCodes.OP_ADD){
+          newBI = new BlockInfoUnderConstruction(
+              newBlock, file.getReplication());
+        } else {
+          // OP_CLOSE should add finalized blocks. This code path
+          // is only executed when loading edits written by prior
+          // versions of Hadoop. Current versions always log
+          // OP_ADD operations as each block is allocated.
+          newBI = new BlockInfo(newBlock, file.getReplication());
+        }
         fsNamesys.getBlockManager().addINode(newBI, file);
         file.addBlock(newBI);
       }

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileUnderConstruction.java

@@ -26,6 +26,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
+import com.google.common.base.Joiner;
+
 /**
  * I-node for file being written.
  */
@@ -94,6 +96,9 @@ public class INodeFileUnderConstruction extends INodeFile {
   // use the modification time as the access time
   //
   INodeFile convertToInodeFile() {
+    assert allBlocksComplete() :
+      "Can't finalize inode " + this + " since it contains " +
+      "non-complete blocks! Blocks are: " + blocksAsString();
     INodeFile obj = new INodeFile(getPermissionStatus(),
                                   getBlocks(),
                                   getReplication(),
@@ -103,6 +108,18 @@ public class INodeFileUnderConstruction extends INodeFile {
     return obj;
     
   }
+  
+  /**
+   * @return true if all of the blocks in this file are marked as completed.
+   */
+  private boolean allBlocksComplete() {
+    for (BlockInfo b : blocks) {
+      if (!b.isComplete()) {
+        return false;
+      }
+    }
+    return true;
+  }
 
   /**
    * Remove a block from the block list. This block should be
@@ -141,4 +158,8 @@ public class INodeFileUnderConstruction extends INodeFile {
     setBlock(numBlocks()-1, ucBlock);
     return ucBlock;
   }
+  
+  private String blocksAsString() {
+    return Joiner.on(",").join(this.blocks);
+  }
 }

+ 73 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPersistBlocks.java

@@ -23,22 +23,34 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
+import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INodeFileUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 
+import java.io.File;
 import java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.List;
 import java.util.Random;
 import static org.junit.Assert.*;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+
 /**
  * A JUnit test for checking if restarting DFS preserves the
  * blocks that are part of an unclosed file.
@@ -57,6 +69,9 @@ public class TestPersistBlocks {
   
   static final byte[] DATA_BEFORE_RESTART = new byte[BLOCK_SIZE * NUM_BLOCKS];
   static final byte[] DATA_AFTER_RESTART = new byte[BLOCK_SIZE * NUM_BLOCKS];
+  
+  private static final String HADOOP_1_0_MULTIBLOCK_TGZ =
+    "hadoop-1.0-multiblock-file.tgz";
   static {
     Random rand = new Random();
     rand.nextBytes(DATA_BEFORE_RESTART);
@@ -277,4 +292,62 @@ public class TestPersistBlocks {
       if (cluster != null) { cluster.shutdown(); }
     }
   }
+  
+  /**
+   * Earlier versions of HDFS didn't persist block allocation to the edit log.
+   * This makes sure that we can still load an edit log when the OP_CLOSE
+   * is the opcode which adds all of the blocks. This is a regression
+   * test for HDFS-2773.
+   * This test uses a tarred pseudo-distributed cluster from Hadoop 1.0
+   * which has a multi-block file. This is similar to the tests in
+   * {@link TestDFSUpgradeFromImage} but none of those images include
+   * a multi-block file.
+   */
+  @Test
+  public void testEarlierVersionEditLog() throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+        
+    String tarFile = System.getProperty("test.cache.data", "build/test/cache")
+      + "/" + HADOOP_1_0_MULTIBLOCK_TGZ;
+    String testDir = System.getProperty("test.build.data", "build/test/data");
+    File dfsDir = new File(testDir, "image-1.0");
+    if (dfsDir.exists() && !FileUtil.fullyDelete(dfsDir)) {
+      throw new IOException("Could not delete dfs directory '" + dfsDir + "'");
+    }
+    FileUtil.unTar(new File(tarFile), new File(testDir));
+
+    File nameDir = new File(dfsDir, "name");
+    GenericTestUtils.assertExists(nameDir);
+    File dataDir = new File(dfsDir, "data");
+    GenericTestUtils.assertExists(dataDir);
+    
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY, nameDir.getAbsolutePath());
+    conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dataDir.getAbsolutePath());
+    
+    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0)
+      .format(false)
+      .manageDataDfsDirs(false)
+      .manageNameDfsDirs(false)
+      .numDataNodes(1)
+      .startupOption(StartupOption.UPGRADE)
+      .build();
+    try {
+      FileSystem fs = cluster.getFileSystem();
+      Path testPath = new Path("/user/todd/4blocks");
+      // Read it without caring about the actual data within - we just need
+      // to make sure that the block states and locations are OK.
+      DFSTestUtil.readFile(fs, testPath);
+      
+      // Ensure that we can append to it - if the blocks were in some funny
+      // state we'd get some kind of issue here. 
+      FSDataOutputStream stm = fs.append(testPath);
+      try {
+        stm.write(1);
+      } finally {
+        IOUtils.closeStream(stm);
+      }
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }

二进制
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-1.0-multiblock-file.tgz