You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2015/04/29 21:43:56 UTC

[28/50] hadoop git commit: HDFS-7994. Detect if resevered EC Block ID is already used during namenode startup. Contributed by Hui Zheng

HDFS-7994. Detect if resevered EC Block ID is already used during namenode startup. Contributed by Hui Zheng


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d5683a3d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d5683a3d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d5683a3d

Branch: refs/heads/HDFS-7285
Commit: d5683a3d4502d666c326f0a30c0c6a11985e0c4f
Parents: d27f5dc
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Thu Apr 16 13:16:37 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Wed Apr 29 11:16:54 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   5 +-
 .../server/blockmanagement/BlockManager.java    |  42 ++++-
 .../hdfs/server/namenode/FSEditLogLoader.java   |   4 +-
 .../hdfs/server/namenode/FSImageFormat.java     |   6 +-
 .../server/namenode/FSImageFormatPBINode.java   |   2 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |   2 +-
 .../server/namenode/TestFSEditLogLoader.java    | 106 ++++++++++++
 .../hdfs/server/namenode/TestFSImage.java       | 169 ++++++++++++++++++-
 8 files changed, 321 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5683a3d/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index b9fc6fa..78ca6d3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -81,4 +81,7 @@
     HDFS-8123. Erasure Coding: Better to move EC related proto messages to a
     separate erasurecoding proto file (Rakesh R via vinayakumarb)
 
-    HDFS-7349. Support DFS command for the EC encoding (vinayakumarb)
\ No newline at end of file
+    HDFS-7349. Support DFS command for the EC encoding (vinayakumarb)
+
+    HDFS-7994. Detect if resevered EC Block ID is already used during namenode
+    startup. (Hui Zheng via szetszwo)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5683a3d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 8006405..dd00e6d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -273,6 +273,9 @@ public class BlockManager {
   /** Check whether name system is running before terminating */
   private boolean checkNSRunning = true;
 
+  /** Check whether there are any non-EC blocks using StripedID */
+  private boolean hasNonEcBlockUsingStripedID = false;
+
   public BlockManager(final Namesystem namesystem, final Configuration conf)
     throws IOException {
     this.namesystem = namesystem;
@@ -2912,6 +2915,24 @@ public class BlockManager {
   }
 
   /**
+   * Get the value of whether there are any non-EC blocks using StripedID.
+   *
+   * @return Returns the value of whether there are any non-EC blocks using StripedID.
+   */
+  public boolean hasNonEcBlockUsingStripedID(){
+    return hasNonEcBlockUsingStripedID;
+  }
+
+  /**
+   * Set the value of whether there are any non-EC blocks using StripedID.
+   *
+   * @param has - the value of whether there are any non-EC blocks using StripedID.
+   */
+  public void hasNonEcBlockUsingStripedID(boolean has){
+    hasNonEcBlockUsingStripedID = has;
+  }
+
+  /**
    * Process a single possibly misreplicated block. This adds it to the
    * appropriate queues if necessary, and returns a result code indicating
    * what happened with it.
@@ -3507,8 +3528,10 @@ public class BlockManager {
     if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
       info = blocksMap.getStoredBlock(
           new Block(BlockIdManager.convertToStripedID(block.getBlockId())));
-    }
-    if (info == null) {
+      if ((info == null) && hasNonEcBlockUsingStripedID()){
+        info = blocksMap.getStoredBlock(block);
+      }
+    } else {
       info = blocksMap.getStoredBlock(block);
     }
     return info;
@@ -3682,6 +3705,21 @@ public class BlockManager {
     return blocksMap.addBlockCollection(block, bc);
   }
 
+  /**
+   * Do some check when adding a block to blocksmap.
+   * For HDFS-7994 to check whether then block is a NonEcBlockUsingStripedID.
+   *
+   */
+  public BlockInfo addBlockCollectionWithCheck(
+      BlockInfo block, BlockCollection bc) {
+    if (!hasNonEcBlockUsingStripedID()){
+      if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
+        hasNonEcBlockUsingStripedID(true);
+      }
+    }
+    return addBlockCollection(block, bc);
+  }
+
   public BlockCollection getBlockCollection(Block b) {
     return blocksMap.getBlockCollection(b);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5683a3d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index de930e6..2c5566c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -991,7 +991,7 @@ public class FSEditLogLoader {
       newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
           file.getBlockReplication());
     }
-    fsNamesys.getBlockManager().addBlockCollection(newBlockInfo, file);
+    fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file);
     file.addBlock(newBlockInfo);
     fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
   }
@@ -1081,7 +1081,7 @@ public class FSEditLogLoader {
           // OP_ADD operations as each block is allocated.
           newBI = new BlockInfoContiguous(newBlock, file.getBlockReplication());
         }
-        fsNamesys.getBlockManager().addBlockCollection(newBI, file);
+        fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);
         file.addBlock(newBI);
         fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5683a3d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index 88b0a97d..704f31e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -700,7 +700,7 @@ public class FSImageFormat {
       if (blocks != null) {
         final BlockManager bm = namesystem.getBlockManager();
         for (int i = 0; i < blocks.length; i++) {
-          file.setBlock(i, bm.addBlockCollection(blocks[i], file));
+          file.setBlock(i, bm.addBlockCollectionWithCheck(blocks[i], file));
         } 
       }
     }
@@ -1005,8 +1005,8 @@ public class FSImageFormat {
         if (oldnode.numBlocks() > 0) {
           BlockInfo ucBlock = cons.getLastBlock();
           // we do not replace the inode, just replace the last block of oldnode
-          BlockInfo info = namesystem.getBlockManager().addBlockCollection(
-              ucBlock, oldnode);
+          BlockInfo info = namesystem.getBlockManager()
+              .addBlockCollectionWithCheck(ucBlock, oldnode);
           oldnode.setBlock(oldnode.numBlocks() - 1, info);
         }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5683a3d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index f293481..3a3f607 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -218,7 +218,7 @@ public final class FSImageFormatPBINode {
       final BlockInfo[] blocks = file.getBlocks();
       if (blocks != null) {
         for (int i = 0; i < blocks.length; i++) {
-          file.setBlock(i, bm.addBlockCollection(blocks[i], file));
+          file.setBlock(i, bm.addBlockCollectionWithCheck(blocks[i], file));
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5683a3d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
index 74baec5..717c8fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
@@ -247,7 +247,7 @@ public class FSImageFormatPBSnapshot {
               (BlockInfoContiguous) fsn.getBlockManager().getStoredBlock(blk);
           if(storedBlock == null) {
             storedBlock = (BlockInfoContiguous) fsn.getBlockManager()
-                .addBlockCollection(new BlockInfoContiguous(blk,
+                .addBlockCollectionWithCheck(new BlockInfoContiguous(blk,
                     copy.getFileReplication()), file);
           }
           blocks[j] = storedBlock;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5683a3d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index c18fd5d..f7a9368 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.junit.Assert.assertFalse;
 import static org.mockito.Mockito.doNothing;
 import static org.mockito.Mockito.spy;
 
@@ -46,6 +47,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -569,4 +571,108 @@ public class TestFSEditLogLoader {
       }
     }
   }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForAddBlock() throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_addblock";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_addblock";
+      String clientMachine = "testMachine_addblock";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      //check whether the hasNonEcBlockUsingStripedID is set
+      //after loading a addblock-editlog
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      fns.getEditLog().logAddBlock(testFilePath, file);
+      file.toCompleteFile(System.currentTimeMillis());
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForUpdateBlocks()
+      throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_002";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser2";
+      String clientMachine = "testMachine1";
+      long blkId = 100;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      file.toCompleteFile(System.currentTimeMillis());
+
+      long newBlkNumBytes = 1024*8;
+      long newTimestamp = 1426222918+3600;
+      file.toUnderConstruction(clientName, clientMachine);
+      file.getLastBlock().setBlockId(-100);
+      file.getLastBlock().setNumBytes(newBlkNumBytes);
+      file.getLastBlock().setGenerationStamp(newTimestamp);
+      fns.getEditLog().logUpdateBlocks(testFilePath, file, true);
+      file.toCompleteFile(System.currentTimeMillis());
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d5683a3d/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index fe130a9..568f49d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -17,11 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
 import java.io.File;
 import java.io.DataOutput;
 import java.io.DataOutputStream;
@@ -62,6 +57,12 @@ import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.junit.Test;
 
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+
 public class TestFSImage {
 
   private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ =
@@ -426,4 +427,162 @@ public class TestFSImage {
       cluster.shutdown();
     }
   }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForLoadFile() throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_loadfile";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_loadfile";
+      String clientMachine = "testMachine_loadfile";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      file.toCompleteFile(System.currentTimeMillis());
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      //after nonEcBlockUsingStripedID is deleted
+      //the hasNonEcBlockUsingStripedID is set to false
+      fs = cluster.getFileSystem();
+      fs.delete(p,false);
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertFalse(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForLoadUCFile()
+      throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_loaducfile";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_loaducfile";
+      String clientMachine = "testMachine_loaducfile";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      Path p = new Path(testFilePath);
+
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testHasNonEcBlockUsingStripedIDForLoadSnapshot()
+      throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/test_block_manager";
+      String testFile = "testfile_loadSnapshot";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser_loadSnapshot";
+      String clientMachine = "testMachine_loadSnapshot";
+      long blkId = -1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+
+      Path d = new Path(testDir);
+      fs.mkdir(d, new FsPermission("755"));
+      fs.allowSnapshot(d);
+
+      Path p = new Path(testFilePath);
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoContiguous cBlk = new BlockInfoContiguous(
+          new Block(blkId, blkNumBytes, timestamp), (short)3);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.addBlock(cBlk);
+      file.toCompleteFile(System.currentTimeMillis());
+
+      fs.createSnapshot(d,"testHasNonEcBlockUsingStripeID");
+      fs.truncate(p,0);
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+      assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }