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 zh...@apache.org on 2015/03/23 21:15:10 UTC

[37/50] [abbrv] hadoop git commit: HDFS-7837. Erasure Coding: allocate and persist striped blocks in NameNode. Contributed by Jing Zhao.

HDFS-7837. Erasure Coding: allocate and persist striped blocks in NameNode. Contributed by Jing Zhao.

 Conflicts:
     hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java


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

Branch: refs/heads/HDFS-7285
Commit: 4b4c7e30082aa0856badb264ffa7c290a92e3895
Parents: f05e27e
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 2 13:44:33 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Mar 23 11:12:35 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockIdManager.java  |  31 +++-
 .../hdfs/server/blockmanagement/BlockInfo.java  |   4 +-
 .../blockmanagement/BlockInfoContiguous.java    |   5 +
 .../blockmanagement/BlockInfoStriped.java       |   8 +-
 .../server/blockmanagement/BlockManager.java    |  44 ++++--
 .../hdfs/server/blockmanagement/BlocksMap.java  |  20 ++-
 .../blockmanagement/DecommissionManager.java    |   9 +-
 .../hdfs/server/namenode/FSDirectory.java       |  27 +++-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  69 ++++++---
 .../hdfs/server/namenode/FSImageFormat.java     |  12 +-
 .../server/namenode/FSImageFormatPBINode.java   |   5 +-
 .../server/namenode/FSImageFormatProtobuf.java  |   9 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  39 ++---
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  25 +++-
 .../server/namenode/NameNodeLayoutVersion.java  |   3 +-
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |   1 +
 .../hdfs/server/namenode/TestAddBlockgroup.java |  85 -----------
 .../server/namenode/TestAddStripedBlocks.java   | 146 +++++++++++++++++++
 18 files changed, 354 insertions(+), 188 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index 3ae54ce..1d69d74 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -103,21 +103,38 @@ public class BlockIdManager {
   }
 
   /**
-   * Sets the maximum allocated block ID for this filesystem. This is
+   * Sets the maximum allocated contiguous block ID for this filesystem. This is
    * the basis for allocating new block IDs.
    */
-  public void setLastAllocatedBlockId(long blockId) {
+  public void setLastAllocatedContiguousBlockId(long blockId) {
     blockIdGenerator.skipTo(blockId);
   }
 
   /**
-   * Gets the maximum sequentially allocated block ID for this filesystem
+   * Gets the maximum sequentially allocated contiguous block ID for this
+   * filesystem
    */
-  public long getLastAllocatedBlockId() {
+  public long getLastAllocatedContiguousBlockId() {
     return blockIdGenerator.getCurrentValue();
   }
 
   /**
+   * Sets the maximum allocated striped block ID for this filesystem. This is
+   * the basis for allocating new block IDs.
+   */
+  public void setLastAllocatedStripedBlockId(long blockId) {
+    blockGroupIdGenerator.skipTo(blockId);
+  }
+
+  /**
+   * Gets the maximum sequentially allocated striped block ID for this
+   * filesystem
+   */
+  public long getLastAllocatedStripedBlockId() {
+    return blockGroupIdGenerator.getCurrentValue();
+  }
+
+  /**
    * Sets the current generation stamp for legacy blocks
    */
   public void setGenerationStampV1(long stamp) {
@@ -188,11 +205,11 @@ public class BlockIdManager {
   /**
    * Increments, logs and then returns the block ID
    */
-  public long nextBlockId() {
+  public long nextContiguousBlockId() {
     return blockIdGenerator.nextValue();
   }
 
-  public long nextBlockGroupId() {
+  public long nextStripedBlockId() {
     return blockGroupIdGenerator.nextValue();
   }
 
@@ -216,7 +233,7 @@ public class BlockIdManager {
     return id < 0;
   }
 
-  public static long convertToGroupID(long id) {
+  public static long convertToStripedID(long id) {
     return id & (~HdfsConstants.BLOCK_GROUP_INDEX_MASK);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index d15cbec..1d8afbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -169,6 +169,8 @@ public abstract class BlockInfo extends Block
    */
   abstract void replaceBlock(BlockInfo newBlock);
 
+  public abstract boolean isStriped();
+
   /**
    * Find specified DatanodeDescriptor.
    * @return index or -1 if not found.
@@ -332,7 +334,7 @@ public abstract class BlockInfo extends Block
   }
 
   static BlockInfo copyOf(BlockInfo b) {
-    if (b instanceof BlockInfoContiguous) {
+    if (!b.isStriped()) {
       return new BlockInfoContiguous((BlockInfoContiguous) b);
     } else {
       return new BlockInfoStriped((BlockInfoStriped) b);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index e54cba3..b5d3d59 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -144,4 +144,9 @@ public class BlockInfoContiguous extends BlockInfo {
     ucBlock.setBlockCollection(getBlockCollection());
     return ucBlock;
   }
+
+  @Override
+  public final boolean isStriped() {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 57de772..8b458df 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
 /**
@@ -57,7 +56,7 @@ public class BlockInfoStriped extends BlockInfo {
     this.setBlockCollection(b.getBlockCollection());
   }
 
-  private short getTotalBlockNum() {
+  short getTotalBlockNum() {
     return (short) (dataBlockNum + parityBlockNum);
   }
 
@@ -175,6 +174,11 @@ public class BlockInfoStriped extends BlockInfo {
   }
 
   @Override
+  public final boolean isStriped() {
+    return true;
+  }
+
+  @Override
   public int numNodes() {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert triplets.length % 3 == 0 : "Malformed BlockInfo";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/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 5cdaa32..634e041 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
@@ -578,11 +578,22 @@ public class BlockManager {
     return maxReplicationStreams;
   }
 
-  /**
-   * @return true if the block has minimum replicas
-   */
-  public boolean checkMinReplication(Block block) {
-    return (countNodes(block).liveReplicas() >= minReplication);
+  public int getDefaultStorageNum(BlockInfo block) {
+    return block.isStriped() ?
+        ((BlockInfoStriped) block).getTotalBlockNum() : defaultReplication;
+  }
+
+  public short getMinStorageNum(BlockInfo block) {
+    return block.isStriped() ?
+        ((BlockInfoStriped) block).getDataBlockNum() : minReplication;
+  }
+
+  public boolean checkMinStorage(BlockInfo block) {
+    return countNodes(block).liveReplicas() >= getMinStorageNum(block);
+  }
+
+  public boolean checkMinStorage(BlockInfo block, int liveNum) {
+    return liveNum >= getMinStorageNum(block);
   }
 
   /**
@@ -626,7 +637,7 @@ public class BlockManager {
       return false; // already completed (e.g. by syncBlock)
     
     final boolean b = commitBlock(lastBlock, commitBlock);
-    if (countNodes(lastBlock).liveReplicas() >= minReplication) {
+    if (checkMinStorage(lastBlock)) {
       completeBlock(bc, bc.numBlocks() - 1, false);
     }
     return b;
@@ -650,7 +661,7 @@ public class BlockManager {
     }
 
     int numNodes = curBlock.numNodes();
-    if (!force && numNodes < minReplication) {
+    if (!force && !checkMinStorage(curBlock, numNodes)) {
       throw new IOException("Cannot complete block: " +
           "block does not satisfy minimal replication requirement.");
     }
@@ -694,9 +705,8 @@ public class BlockManager {
    * when tailing edit logs as a Standby.
    */
   public BlockInfo forceCompleteBlock(final BlockCollection bc,
-      final BlockInfoContiguousUnderConstruction block) throws IOException {
-    // TODO: support BlockInfoStripedUC for editlog
-    block.commitBlock(block);
+      final BlockInfo block) throws IOException {
+    BlockInfo.commitBlock(block, block);
     return completeBlock(bc, block, true);
   }
 
@@ -747,7 +757,7 @@ public class BlockManager {
     // count in safe-mode.
     namesystem.adjustSafeModeBlockTotals(
         // decrement safe if we had enough
-        targets.length >= minReplication ? -1 : 0,
+        checkMinStorage(oldBlock, targets.length) ? -1 : 0,
         // always decrement total blocks
         -1);
 
@@ -1191,8 +1201,8 @@ public class BlockManager {
     NumberReplicas numberOfReplicas = countNodes(b.stored);
     boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >= bc
         .getBlockReplication();
-    boolean minReplicationSatisfied =
-        numberOfReplicas.liveReplicas() >= minReplication;
+    boolean minReplicationSatisfied = checkMinStorage(b.stored,
+        numberOfReplicas.liveReplicas());
     boolean hasMoreCorruptReplicas = minReplicationSatisfied &&
         (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) >
         bc.getBlockReplication();
@@ -2446,7 +2456,7 @@ public class BlockManager {
     // Now check for completion of blocks and safe block count
     int numCurrentReplica = countLiveNodes(storedBlock);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
-        && numCurrentReplica >= minReplication) {
+        && checkMinStorage(storedBlock, numCurrentReplica)) {
       completeBlock(storedBlock.getBlockCollection(), storedBlock, false);
     } else if (storedBlock.isComplete()) {
       // check whether safe replication is reached for the block
@@ -2521,7 +2531,7 @@ public class BlockManager {
       + pendingReplications.getNumReplicas(storedBlock);
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
-        numLiveReplicas >= minReplication) {
+        checkMinStorage(storedBlock, numLiveReplicas)) {
       storedBlock = completeBlock(bc, storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
@@ -3192,6 +3202,8 @@ public class BlockManager {
   /**
    * Return the number of nodes hosting a given block, grouped
    * by the state of those replicas.
+   * For a striped block, this includes nodes storing blocks belonging to the
+   * striped block group.
    */
   public NumberReplicas countNodes(Block b) {
     int decommissioned = 0;
@@ -3340,7 +3352,7 @@ public class BlockManager {
     BlockInfo info = null;
     if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
       info = blocksMap.getStoredBlock(
-          new Block(BlockIdManager.convertToGroupID(block.getBlockId())));
+          new Block(BlockIdManager.convertToStripedID(block.getBlockId())));
     }
     if (info == null) {
       info = blocksMap.getStoredBlock(block);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
index d383de8..0cf5fe6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
@@ -43,8 +43,15 @@ class BlocksMap {
 
     @Override
     public boolean hasNext() {
-      return blockInfo != null && nextIdx < blockInfo.getCapacity()
-              && blockInfo.getDatanode(nextIdx) != null;
+      if (blockInfo == null) {
+        return false;
+      }
+      while (nextIdx < blockInfo.getCapacity() &&
+          blockInfo.getDatanode(nextIdx) == null) {
+        // note that for striped blocks there may be null in the triplets
+        nextIdx++;
+      }
+      return nextIdx < blockInfo.getCapacity();
     }
 
     @Override
@@ -123,10 +130,13 @@ class BlocksMap {
       return;
 
     blockInfo.setBlockCollection(null);
-    // TODO: fix this logic for block group
-    for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
+    final int size = blockInfo instanceof BlockInfoContiguous ?
+        blockInfo.numNodes() : blockInfo.getCapacity();
+    for(int idx = size - 1; idx >= 0; idx--) {
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
-      dn.removeBlock(blockInfo); // remove from the list and wipe the location
+      if (dn != null) {
+        dn.removeBlock(blockInfo); // remove from the list and wipe the location
+      }
     }
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 9355329..0faf3ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -545,7 +545,7 @@ public class DecommissionManager {
       int underReplicatedInOpenFiles = 0;
       while (it.hasNext()) {
         numBlocksChecked++;
-        final BlockInfoContiguous block = it.next();
+        final BlockInfo block = it.next();
         // Remove the block from the list if it's no longer in the block map,
         // e.g. the containing file has been deleted
         if (blockManager.blocksMap.getStoredBlock(block) == null) {
@@ -579,8 +579,9 @@ public class DecommissionManager {
         }
 
         // Even if the block is under-replicated, 
-        // it doesn't block decommission if it's sufficiently replicated 
-        if (isSufficientlyReplicated(block, bc, num)) {
+        // it doesn't block decommission if it's sufficiently replicated
+        BlockInfoContiguous blk = (BlockInfoContiguous) block;
+        if (isSufficientlyReplicated(blk, bc, num)) {
           if (pruneSufficientlyReplicated) {
             it.remove();
           }
@@ -589,7 +590,7 @@ public class DecommissionManager {
 
         // We've found an insufficiently replicated block.
         if (insufficientlyReplicated != null) {
-          insufficientlyReplicated.add(block);
+          insufficientlyReplicated.add(blk);
         }
         // Log if this is our first time through
         if (firstReplicationLog) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index e9808d3..f7a9c9e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
@@ -411,6 +412,10 @@ public class FSDirectory implements Closeable {
     writeLock();
     try {
       newiip = addINode(existing, newNode);
+      // TODO: we will no longer use storage policy for "Erasure Coding Zone"
+      if (newiip != null && newNode.isStriped()) {
+        newNode.addStripedBlocksFeature();
+      }
     } finally {
       writeUnlock();
     }
@@ -446,6 +451,10 @@ public class FSDirectory implements Closeable {
     try {
       INodesInPath iip = addINode(existing, newNode);
       if (iip != null) {
+        // TODO: we will no longer use storage policy for "Erasure Coding Zone"
+        if (newNode.isStriped()) {
+          newNode.addStripedBlocksFeature();
+        }
         if (aclEntries != null) {
           AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID);
         }
@@ -467,7 +476,7 @@ public class FSDirectory implements Closeable {
   /**
    * Add a block to the file. Returns a reference to the added block.
    */
-  BlockInfoContiguous addBlock(String path, INodesInPath inodesInPath,
+  BlockInfo addBlock(String path, INodesInPath inodesInPath,
       Block block, DatanodeStorageInfo[] targets, boolean isStriped)
       throws IOException {
     writeLock();
@@ -479,16 +488,20 @@ public class FSDirectory implements Closeable {
       Preconditions.checkState(fileINode.isUnderConstruction());
 
       // check quota limits and updated space consumed
+      // TODO add quota usage for EC files
       updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
           fileINode.getBlockReplication(), true);
 
       // associate new last block for the file
-      BlockInfoContiguousUnderConstruction blockInfo =
-        new BlockInfoContiguousUnderConstruction(
-            block,
-            numLocations,
-            BlockUCState.UNDER_CONSTRUCTION,
-            targets);
+      final BlockInfo blockInfo;
+      if (isStriped) {
+        blockInfo = new BlockInfoStripedUnderConstruction(block,
+            HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS,
+            BlockUCState.UNDER_CONSTRUCTION, targets);
+      } else {
+        blockInfo = new BlockInfoContiguousUnderConstruction(block,
+            numLocations, BlockUCState.UNDER_CONSTRUCTION, targets);
+      }
       getBlockManager().addBlockCollection(blockInfo, fileINode);
       fileINode.addBlock(blockInfo);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/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 e0c823e..f9dc542 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
@@ -35,7 +35,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -414,7 +416,8 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      updateBlocks(fsDir, addCloseOp, iip, newFile);
+      // TODO whether the file is striped should later be retrieved from iip
+      updateBlocks(fsDir, addCloseOp, iip, newFile, newFile.isStriped());
       break;
     }
     case OP_CLOSE: {
@@ -434,7 +437,8 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      updateBlocks(fsDir, addCloseOp, iip, file);
+      // TODO whether the file is striped should later be retrieved from iip
+      updateBlocks(fsDir, addCloseOp, iip, file, file.isStriped());
 
       // Now close the file
       if (!file.isUnderConstruction() &&
@@ -492,7 +496,8 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // Update in-memory data structures
-      updateBlocks(fsDir, updateOp, iip, oldFile);
+      // TODO whether the file is striped should later be retrieved from iip
+      updateBlocks(fsDir, updateOp, iip, oldFile, oldFile.isStriped());
       
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
@@ -508,7 +513,8 @@ public class FSEditLogLoader {
       }
       INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
       // add the new block to the INodeFile
-      addNewBlock(addBlockOp, oldFile);
+      // TODO whether the file is striped should later be retrieved from iip
+      addNewBlock(addBlockOp, oldFile, oldFile.isStriped());
       break;
     }
     case OP_SET_REPLICATION: {
@@ -784,8 +790,15 @@ public class FSEditLogLoader {
     }
     case OP_ALLOCATE_BLOCK_ID: {
       AllocateBlockIdOp allocateBlockIdOp = (AllocateBlockIdOp) op;
-      fsNamesys.getBlockIdManager().setLastAllocatedBlockId(
-          allocateBlockIdOp.blockId);
+      if (BlockIdManager.isStripedBlockID(allocateBlockIdOp.blockId)) {
+        // ALLOCATE_BLOCK_ID is added for sequential block id, thus if the id
+        // is negative, it must belong to striped blocks
+        fsNamesys.getBlockIdManager().setLastAllocatedStripedBlockId(
+            allocateBlockIdOp.blockId);
+      } else {
+        fsNamesys.getBlockIdManager().setLastAllocatedContiguousBlockId(
+            allocateBlockIdOp.blockId);
+      }
       break;
     }
     case OP_ROLLING_UPGRADE_START: {
@@ -937,9 +950,9 @@ public class FSEditLogLoader {
 
   /**
    * Add a new block into the given INodeFile
-   * TODO support adding striped block
    */
-  private void addNewBlock(AddBlockOp op, INodeFile file) throws IOException {
+  private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped)
+      throws IOException {
     BlockInfo[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
     Block newBlock= op.getLastBlock();
@@ -947,7 +960,7 @@ public class FSEditLogLoader {
     if (pBlock != null) { // the penultimate block is not null
       assert oldBlocks != null && oldBlocks.length > 0;
       // compare pBlock with the last block of oldBlocks
-      Block oldLastBlock = oldBlocks[oldBlocks.length - 1];
+      BlockInfo oldLastBlock = oldBlocks[oldBlocks.length - 1];
       if (oldLastBlock.getBlockId() != pBlock.getBlockId()
           || oldLastBlock.getGenerationStamp() != pBlock.getGenerationStamp()) {
         throw new IOException(
@@ -957,29 +970,33 @@ public class FSEditLogLoader {
       }
       
       oldLastBlock.setNumBytes(pBlock.getNumBytes());
-      if (oldLastBlock instanceof BlockInfoContiguousUnderConstruction) {
-        fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoContiguousUnderConstruction) oldLastBlock);
+      if (!oldLastBlock.isComplete()) {
+        fsNamesys.getBlockManager().forceCompleteBlock(file, oldLastBlock);
         fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock);
       }
     } else { // the penultimate block is null
       Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
     }
     // add the new block
-    BlockInfoContiguous newBI = new BlockInfoContiguousUnderConstruction(
-          newBlock, file.getBlockReplication());
-    fsNamesys.getBlockManager().addBlockCollection(newBI, file);
-    file.addBlock(newBI);
+    final BlockInfo newBlockInfo;
+    if (isStriped) {
+      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
+          HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+    } else {
+      newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
+          file.getBlockReplication());
+    }
+    fsNamesys.getBlockManager().addBlockCollection(newBlockInfo, file);
+    file.addBlock(newBlockInfo);
     fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
   }
   
   /**
    * Update in-memory data structures with new block information.
-   * TODO support adding striped block
    * @throws IOException
    */
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
-      INodesInPath iip, INodeFile file) throws IOException {
+      INodesInPath iip, INodeFile file, boolean isStriped) throws IOException {
     // Update its block list
     BlockInfo[] oldBlocks = file.getBlocks();
     Block[] newBlocks = op.getBlocks();
@@ -1008,11 +1025,10 @@ public class FSEditLogLoader {
         oldBlock.getGenerationStamp() != newBlock.getGenerationStamp();
       oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
       
-      if (oldBlock instanceof BlockInfoContiguousUnderConstruction &&
+      if (!oldBlock.isComplete() &&
           (!isLastBlock || op.shouldCompleteLastBlock())) {
         changeMade = true;
-        fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoContiguousUnderConstruction) oldBlock);
+        fsNamesys.getBlockManager().forceCompleteBlock(file, oldBlock);
       }
       if (changeMade) {
         // The state or gen-stamp of the block has changed. So, we may be
@@ -1041,13 +1057,18 @@ public class FSEditLogLoader {
       // We're adding blocks
       for (int i = oldBlocks.length; i < newBlocks.length; i++) {
         Block newBlock = newBlocks[i];
-        BlockInfoContiguous newBI;
+        final BlockInfo newBI;
         if (!op.shouldCompleteLastBlock()) {
           // TODO: shouldn't this only be true for the last block?
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
-          newBI = new BlockInfoContiguousUnderConstruction(
-              newBlock, file.getBlockReplication());
+          if (isStriped) {
+            newBI = new BlockInfoStripedUnderConstruction(newBlock,
+                HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+          } else {
+            newBI = new BlockInfoContiguousUnderConstruction(newBlock,
+                file.getBlockReplication());
+          }
         } else {
           // OP_CLOSE should add finalized blocks. This code path
           // is only executed when loading edits written by prior

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/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 d62b804..2e6e741 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
@@ -357,7 +357,14 @@ public class FSImageFormat {
 
           // read the max sequential block ID.
           long maxSequentialBlockId = in.readLong();
-          namesystem.getBlockIdManager().setLastAllocatedBlockId(maxSequentialBlockId);
+          namesystem.getBlockIdManager().setLastAllocatedContiguousBlockId(
+              maxSequentialBlockId);
+          if (NameNodeLayoutVersion.supports(
+              NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)) {
+            final long maxStripedBlockId = in.readLong();
+            namesystem.getBlockIdManager().setLastAllocatedStripedBlockId(
+                maxStripedBlockId);
+          }
         } else {
 
           long startingGenStamp = namesystem.getBlockIdManager()
@@ -1264,7 +1271,8 @@ public class FSImageFormat {
         out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampV1());
         out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampV2());
         out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampAtblockIdSwitch());
-        out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedBlockId());
+        out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedContiguousBlockId());
+        out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedStripedBlockId());
         out.writeLong(context.getTxId());
         out.writeLong(sourceNamesystem.dir.getLastInodeId());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/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 a025bb0..5627788 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
@@ -630,8 +630,9 @@ public final class FSImageFormatPBINode {
       INodeSection.INodeFile.Builder b = buildINodeFile(n,
           parent.getSaverContext());
 
-      if (n.getBlocks() != null) {
-        for (Block block : n.getBlocks()) {
+      BlockInfoContiguous[] cBlks = n.getContiguousBlocks();
+      if (cBlks != null) {
+        for (Block block : cBlks) {
           b.addBlocks(PBHelper.convert(block));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
index 3ee848a..62dc068 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
@@ -296,7 +296,11 @@ public final class FSImageFormatProtobuf {
       blockIdManager.setGenerationStampV1(s.getGenstampV1());
       blockIdManager.setGenerationStampV2(s.getGenstampV2());
       blockIdManager.setGenerationStampV1Limit(s.getGenstampV1Limit());
-      blockIdManager.setLastAllocatedBlockId(s.getLastAllocatedBlockId());
+      blockIdManager.setLastAllocatedContiguousBlockId(s.getLastAllocatedBlockId());
+      if (s.hasLastAllocatedStripedBlockId()) {
+        blockIdManager.setLastAllocatedStripedBlockId(
+            s.getLastAllocatedStripedBlockId());
+      }
       imgTxId = s.getTransactionId();
       if (s.hasRollingUpgradeStartTime()
           && fsn.getFSImage().hasRollbackFSImage()) {
@@ -536,7 +540,8 @@ public final class FSImageFormatProtobuf {
           .setGenstampV1(blockIdManager.getGenerationStampV1())
           .setGenstampV1Limit(blockIdManager.getGenerationStampV1Limit())
           .setGenstampV2(blockIdManager.getGenerationStampV2())
-          .setLastAllocatedBlockId(blockIdManager.getLastAllocatedBlockId())
+          .setLastAllocatedBlockId(blockIdManager.getLastAllocatedContiguousBlockId())
+          .setLastAllocatedStripedBlockId(blockIdManager.getLastAllocatedStripedBlockId())
           .setTransactionId(context.getTxId());
 
       // We use the non-locked version of getNamespaceInfo here since

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 88e946b..b56eab5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -206,7 +206,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -2052,7 +2051,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean shouldRecoverNow = (newBlock == null);
 
     BlockInfo oldBlock = file.getLastBlock();
-    assert oldBlock instanceof BlockInfoContiguous;
+    assert !oldBlock.isStriped();
 
     boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file,
         (BlockInfoContiguous) oldBlock);
@@ -3066,6 +3065,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // choose targets for the new block to be allocated.
+    // TODO we need block placement policy for striped block groups (HDFS-7613)
     final DatanodeStorageInfo targets[] = getBlockManager().chooseTarget4NewBlock( 
         src, numTargets, clientNode, excludedNodes, blockSize, favoredNodes,
         storagePolicyID);
@@ -3106,8 +3106,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       // allocate new block, record block locations in INode.
       newBlock = createNewBlock(isStriped);
-      INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
-      saveAllocatedBlock(src, inodesInPath, newBlock, targets, isStriped);
+      saveAllocatedBlock(src, fileState.iip, newBlock, targets, isStriped);
 
       persistNewBlock(src, pendingFile);
       offset = pendingFile.computeFileSize();
@@ -3530,13 +3529,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param isStriped is the file under striping or contigunous layout?
    * @throws QuotaExceededException If addition of block exceeds space quota
    */
-  // TODO: support striped block
-  BlockInfoContiguous saveAllocatedBlock(String src, INodesInPath inodesInPath,
+  BlockInfo saveAllocatedBlock(String src, INodesInPath inodesInPath,
       Block newBlock, DatanodeStorageInfo[] targets, boolean isStriped)
-          throws IOException {
+      throws IOException {
     assert hasWriteLock();
-    BlockInfoContiguous b = dir.addBlock(src, inodesInPath, newBlock, targets,
-        isStriped);
+    BlockInfo b = dir.addBlock(src, inodesInPath, newBlock, targets, isStriped);
     NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
     return b;
@@ -3586,13 +3583,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private boolean isCompleteBlock(String src, BlockInfo b) {
     if (!b.isComplete()) {
       final int numNodes = b.numNodes();
-      final int min;
+      final int min = blockManager.getMinStorageNum(b);
       final BlockUCState state = b.getBlockUCState();
-      if (b instanceof BlockInfoStripedUnderConstruction) {
-        min = ((BlockInfoStripedUnderConstruction) b).getDataBlockNum();
-      } else {
-        min = blockManager.minReplication;
-      }
       LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " + state
           + ", replication# = " + numNodes + (numNodes < min ? " < " : " >= ")
           + " minimum = " + min + ") in file " + src);
@@ -3783,7 +3775,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         BlockInfo bi = getStoredBlock(b);
         if (bi.isComplete()) {
           numRemovedComplete++;
-          if (bi.numNodes() >= blockManager.minReplication) {
+          if (blockManager.checkMinStorage(bi, bi.numNodes())) {
             numRemovedSafe++;
           }
         }
@@ -4012,7 +4004,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       curBlock = blocks[nrCompleteBlocks];
       if(!curBlock.isComplete())
         break;
-      assert blockManager.checkMinReplication(curBlock) :
+      assert blockManager.checkMinStorage(curBlock) :
               "A COMPLETE block is not minimally replicated in " + src;
     }
 
@@ -4047,8 +4039,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
 
     // If penultimate block doesn't exist then its minReplication is met
-    boolean penultimateBlockMinReplication = penultimateBlock == null ||
-        blockManager.checkMinReplication(penultimateBlock);
+    boolean penultimateBlockMinStorage = penultimateBlock == null ||
+        blockManager.checkMinStorage(penultimateBlock);
 
     switch(lastBlockState) {
     case COMPLETE:
@@ -4056,8 +4048,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       break;
     case COMMITTED:
       // Close file if committed blocks are minimally replicated
-      if(penultimateBlockMinReplication &&
-          blockManager.checkMinReplication(lastBlock)) {
+      if(penultimateBlockMinStorage &&
+          blockManager.checkMinStorage(lastBlock)) {
         finalizeINodeFileUnderConstruction(src, pendingFile,
             iip.getLatestSnapshotId());
         NameNode.stateChangeLog.warn("BLOCK*"
@@ -4157,6 +4149,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // Adjust disk space consumption if required
+    // TODO: support EC files
     final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();    
     if (diff > 0) {
       try {
@@ -5043,8 +5036,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * Persist the new block (the last block of the given file).
-   * @param path
-   * @param file
    */
   private void persistNewBlock(String path, INodeFile file) {
     Preconditions.checkArgument(file.isUnderConstruction());
@@ -6160,7 +6151,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     assert hasWriteLock();
     checkNameNodeSafeMode("Cannot get next block ID");
     final long blockId = isStriped ?
-        blockIdManager.nextBlockGroupId() : blockIdManager.nextBlockId();
+        blockIdManager.nextStripedBlockId() : blockIdManager.nextContiguousBlockId();
     getEditLog().logAllocateBlockId(blockId);
     // NB: callers sync the log
     return blockId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index d3c5e3e..2906996 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -86,7 +86,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   static enum HeaderFormat {
     PREFERRED_BLOCK_SIZE(null, 48, 1),
-    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 1),
+    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 0),
     STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
         0);
 
@@ -261,10 +261,10 @@ public class INodeFile extends INodeWithAdditionalFields
   public void setBlock(int index, BlockInfo blk) {
     FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
     if (sb == null) {
-      assert blk instanceof BlockInfoContiguous;
+      assert !blk.isStriped();
       this.blocks[index] = (BlockInfoContiguous) blk;
     } else {
-      assert blk instanceof BlockInfoStriped;
+      assert blk.isStriped();
       assert hasNoContiguousBlock();
       sb.setBlock(index, (BlockInfoStriped) blk);
     }
@@ -282,12 +282,12 @@ public class INodeFile extends INodeWithAdditionalFields
     final BlockInfo ucBlock;
     FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
     if (sb == null) {
-      assert lastBlock instanceof BlockInfoContiguous;
+      assert !lastBlock.isStriped();
       ucBlock = ((BlockInfoContiguous) lastBlock)
           .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
     } else {
       assert hasNoContiguousBlock();
-      assert lastBlock instanceof BlockInfoStriped;
+      assert lastBlock.isStriped();
       ucBlock = ((BlockInfoStriped) lastBlock)
           .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
     }
@@ -546,7 +546,7 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * add a contiguous block to the block list
    */
-  void addBlock(BlockInfoContiguous newblock) {
+  private void addContiguousBlock(BlockInfoContiguous newblock) {
     if (this.blocks == null) {
       this.setContiguousBlocks(new BlockInfoContiguous[]{newblock});
     } else {
@@ -558,6 +558,19 @@ public class INodeFile extends INodeWithAdditionalFields
     }
   }
 
+  /** add a striped or contiguous block */
+  void addBlock(BlockInfo newblock) {
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      assert !newblock.isStriped();
+      addContiguousBlock((BlockInfoContiguous) newblock);
+    } else {
+      assert newblock.isStriped();
+      assert hasNoContiguousBlock();
+      sb.addBlock((BlockInfoStriped) newblock);
+    }
+  }
+
   /** Set the blocks. */
   public void setContiguousBlocks(BlockInfoContiguous[] blocks) {
     this.blocks = blocks;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
index d235e2b..f93218f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
@@ -72,7 +72,8 @@ public class NameNodeLayoutVersion {
     BLOCK_STORAGE_POLICY(-60, "Block Storage policy"),
     TRUNCATE(-61, "Truncate"),
     APPEND_NEW_BLOCK(-62, "Support appending to new block"),
-    QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types");
+    QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types"),
+    ERASURE_CODING(-64, "Support erasure coding");
 
     private final FeatureInfo info;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
index b6fd033..3f3a71e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
@@ -73,6 +73,7 @@ message NameSystemSection {
   optional uint64 lastAllocatedBlockId = 5;
   optional uint64 transactionId = 6;
   optional uint64 rollingUpgradeStartTime = 7;
+  optional uint64 lastAllocatedStripedBlockId = 8;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
deleted file mode 100644
index a2ef7b2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestAddBlockgroup {
-
-  public static final Log LOG = LogFactory.getLog(TestAddBlockgroup.class);
-
-  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
-      HdfsConstants.NUM_PARITY_BLOCKS;
-  private final short NUM_DATANODES = GROUP_SIZE;
-
-  private static final int BLOCKSIZE = 1024;
-  private static final short REPLICATION = 3;
-
-  private MiniDFSCluster cluster;
-  private Configuration conf;
-
-  @Before
-  public void setup() throws IOException {
-    conf = new Configuration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES)
-        .build();
-    cluster.waitActive();
-    cluster.getFileSystem().setStoragePolicy(new Path("/"),
-        HdfsConstants.EC_STORAGE_POLICY_NAME);
-  }
-
-  @After
-  public void tearDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testAddBlockGroup() throws Exception {
-    DistributedFileSystem fs = cluster.getFileSystem();
-    FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
-
-    final Path file1 = new Path("/file1");
-    DFSTestUtil.createFile(fs, file1, BLOCKSIZE * 2, REPLICATION, 0L);
-    INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
-    BlockInfo[] file1Blocks = file1Node.getBlocks();
-    assertEquals(2, file1Blocks.length);
-    assertEquals(GROUP_SIZE, file1Blocks[0].numNodes());
-    assertEquals(HdfsConstants.MAX_BLOCKS_IN_GROUP,
-        file1Blocks[1].getBlockId() - file1Blocks[0].getBlockId());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4b4c7e30/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
new file mode 100644
index 0000000..7226f51
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestAddStripedBlocks {
+  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
+      HdfsConstants.NUM_PARITY_BLOCKS;
+
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem dfs;
+
+  @Before
+  public void setup() throws IOException {
+    cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
+        .numDataNodes(GROUP_SIZE).build();
+    cluster.waitActive();
+    dfs = cluster.getFileSystem();
+    dfs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testAddStripedBlock() throws Exception {
+    final Path file = new Path("/file1");
+    // create an empty file
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1);
+
+      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+      LocatedBlock newBlock = cluster.getNamesystem().getAdditionalBlock(
+          file.toString(), fileNode.getId(), dfs.getClient().getClientName(),
+          null, null, null);
+      assertEquals(GROUP_SIZE, newBlock.getLocations().length);
+      assertEquals(GROUP_SIZE, newBlock.getStorageIDs().length);
+
+      BlockInfo[] blocks = fileNode.getBlocks();
+      assertEquals(1, blocks.length);
+      Assert.assertTrue(blocks[0].isStriped());
+
+      checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), true);
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+
+    // restart NameNode to check editlog
+    cluster.restartNameNode(true);
+    FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+    INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+    BlockInfo[] blocks = fileNode.getBlocks();
+    assertEquals(1, blocks.length);
+    Assert.assertTrue(blocks[0].isStriped());
+    checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
+
+    // save namespace, restart namenode, and check
+    dfs = cluster.getFileSystem();
+    dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    dfs.saveNamespace();
+    dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    cluster.restartNameNode(true);
+    fsdir = cluster.getNamesystem().getFSDirectory();
+    fileNode = fsdir.getINode4Write(file.toString()).asFile();
+    blocks = fileNode.getBlocks();
+    assertEquals(1, blocks.length);
+    Assert.assertTrue(blocks[0].isStriped());
+    checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
+  }
+
+  private void checkStripedBlockUC(BlockInfoStriped block,
+      boolean checkReplica) {
+    assertEquals(0, block.numNodes());
+    Assert.assertFalse(block.isComplete());
+    Assert.assertEquals(HdfsConstants.NUM_DATA_BLOCKS, block.getDataBlockNum());
+    Assert.assertEquals(HdfsConstants.NUM_PARITY_BLOCKS,
+        block.getParityBlockNum());
+    Assert.assertEquals(0,
+        block.getBlockId() & HdfsConstants.BLOCK_GROUP_INDEX_MASK);
+
+    final BlockInfoStripedUnderConstruction blockUC =
+        (BlockInfoStripedUnderConstruction) block;
+    Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+        blockUC.getBlockUCState());
+    if (checkReplica) {
+      Assert.assertEquals(GROUP_SIZE, blockUC.getNumExpectedLocations());
+      DatanodeStorageInfo[] storages = blockUC.getExpectedStorageLocations();
+      for (DataNode dn : cluster.getDataNodes()) {
+        Assert.assertTrue(includeDataNode(dn.getDatanodeId(), storages));
+      }
+    }
+  }
+
+  private boolean includeDataNode(DatanodeID dn, DatanodeStorageInfo[] storages) {
+    for (DatanodeStorageInfo storage : storages) {
+      if (storage.getDatanodeDescriptor().equals(dn)) {
+        return true;
+      }
+    }
+    return false;
+  }
+}