浏览代码

HDFS-8367 BlockInfoStriped uses EC schema. Contributed by Kai Sasaki

Kai Zheng 10 年之前
父节点
当前提交
c9103e9cac
共有 15 个文件被更改,包括 99 次插入80 次删除
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
  2. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  3. 16 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
  4. 6 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
  5. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
  6. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  7. 18 16
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  8. 6 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  9. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  10. 8 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  11. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  12. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
  13. 6 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
  14. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
  15. 12 27
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java

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

@@ -215,3 +215,5 @@
 
 
     HDFS-8391. NN should consider current EC tasks handling count from DN while 
     HDFS-8391. NN should consider current EC tasks handling count from DN while 
     assigning new tasks. (umamahesh)
     assigning new tasks. (umamahesh)
+
+    HDFS-8367. BlockInfoStriped uses EC schema. (Kai Sasaki via Kai Zheng)

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -203,6 +203,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BalancerBandwidthCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand;
@@ -445,9 +446,8 @@ public class PBHelper {
     return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
     return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
   }
   }
 
 
-  public static BlockInfoStriped convert(StripedBlockProto p) {
-    return new BlockInfoStriped(convert(p.getBlock()),
-        (short) p.getDataBlockNum(), (short) p.getParityBlockNum());
+  public static BlockInfoStriped convert(StripedBlockProto p, ECSchema schema) {
+    return new BlockInfoStriped(convert(p.getBlock()), schema);
   }
   }
 
 
   public static StripedBlockProto convert(BlockInfoStriped blk) {
   public static StripedBlockProto convert(BlockInfoStriped blk) {

+ 16 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java

@@ -19,7 +19,9 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 
 
@@ -39,6 +41,7 @@ import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_S
 public class BlockInfoStriped extends BlockInfo {
 public class BlockInfoStriped extends BlockInfo {
   private final short dataBlockNum;
   private final short dataBlockNum;
   private final short parityBlockNum;
   private final short parityBlockNum;
+  private final ECSchema schema;
   /**
   /**
    * Always the same size with triplets. Record the block index for each triplet
    * Always the same size with triplets. Record the block index for each triplet
    * TODO: actually this is only necessary for over-replicated block. Thus can
    * TODO: actually this is only necessary for over-replicated block. Thus can
@@ -46,16 +49,17 @@ public class BlockInfoStriped extends BlockInfo {
    */
    */
   private byte[] indices;
   private byte[] indices;
 
 
-  public BlockInfoStriped(Block blk, short dataBlockNum, short parityBlockNum) {
-    super(blk, (short) (dataBlockNum + parityBlockNum));
-    indices = new byte[dataBlockNum + parityBlockNum];
+  public BlockInfoStriped(Block blk, ECSchema schema) {
+    super(blk, (short) (schema.getNumDataUnits() + schema.getNumParityUnits()));
+    indices = new byte[schema.getNumDataUnits() + schema.getNumParityUnits()];
     initIndices();
     initIndices();
-    this.dataBlockNum = dataBlockNum;
-    this.parityBlockNum = parityBlockNum;
+    this.schema = schema;
+    this.dataBlockNum = (short)schema.getNumDataUnits();
+    this.parityBlockNum = (short)schema.getNumParityUnits();
   }
   }
 
 
   BlockInfoStriped(BlockInfoStriped b) {
   BlockInfoStriped(BlockInfoStriped b) {
-    this(b, b.dataBlockNum, b.parityBlockNum);
+    this(b, b.getSchema());
     this.setBlockCollection(b.getBlockCollection());
     this.setBlockCollection(b.getBlockCollection());
   }
   }
 
 
@@ -71,6 +75,10 @@ public class BlockInfoStriped extends BlockInfo {
     return parityBlockNum;
     return parityBlockNum;
   }
   }
 
 
+  public ECSchema getSchema() {
+    return schema;
+  }
+
   private void initIndices() {
   private void initIndices() {
     for (int i = 0; i < indices.length; i++) {
     for (int i = 0; i < indices.length; i++) {
       indices[i] = -1;
       indices[i] = -1;
@@ -231,8 +239,8 @@ public class BlockInfoStriped extends BlockInfo {
       BlockUCState s, DatanodeStorageInfo[] targets) {
       BlockUCState s, DatanodeStorageInfo[] targets) {
     final BlockInfoStripedUnderConstruction ucBlock;
     final BlockInfoStripedUnderConstruction ucBlock;
     if(isComplete()) {
     if(isComplete()) {
-      ucBlock = new BlockInfoStripedUnderConstruction(this, getDataBlockNum(),
-              getParityBlockNum(), s, targets);
+      ucBlock = new BlockInfoStripedUnderConstruction(this, schema,
+          s, targets);
       ucBlock.setBlockCollection(getBlockCollection());
       ucBlock.setBlockCollection(getBlockCollection());
     } else {
     } else {
       // the block is already under construction
       // the block is already under construction

+ 6 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java

@@ -21,6 +21,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
@@ -56,17 +57,16 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
   /**
   /**
    * Constructor with null storage targets.
    * Constructor with null storage targets.
    */
    */
-  public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum,
-      short parityBlockNum) {
-    this(blk, dataBlockNum, parityBlockNum, UNDER_CONSTRUCTION, null);
+  public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema) {
+    this(blk, schema, UNDER_CONSTRUCTION, null);
   }
   }
 
 
   /**
   /**
    * Create a striped block that is currently being constructed.
    * Create a striped block that is currently being constructed.
    */
    */
-  public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum,
-      short parityBlockNum, BlockUCState state, DatanodeStorageInfo[] targets) {
-    super(blk, dataBlockNum, parityBlockNum);
+  public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema,
+      BlockUCState state, DatanodeStorageInfo[] targets) {
+    super(blk, schema);
     assert getBlockUCState() != COMPLETE :
     assert getBlockUCState() != COMPLETE :
       "BlockInfoStripedUnderConstruction cannot be in COMPLETE state";
       "BlockInfoStripedUnderConstruction cannot be in COMPLETE state";
     this.blockUCState = state;
     this.blockUCState = state;

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

@@ -532,8 +532,8 @@ class FSDirWriteFileOp {
         // check quota limits and updated space consumed
         // check quota limits and updated space consumed
         fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
         fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
             numLocations, true);
             numLocations, true);
-        blockInfo = new BlockInfoStripedUnderConstruction(block, numDataUnits,
-            numParityUnits, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+        blockInfo = new BlockInfoStripedUnderConstruction(block, ecSchema,
+            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
             targets);
             targets);
       } else {
       } else {
         // check quota limits and updated space consumed
         // check quota limits and updated space consumed

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -54,6 +54,9 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 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.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;

+ 18 - 16
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -104,6 +104,7 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgress.Counter;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
 import org.apache.hadoop.hdfs.server.namenode.startupprogress.Step;
 import org.apache.hadoop.hdfs.util.Holder;
 import org.apache.hadoop.hdfs.util.Holder;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.ChunkedArrayList;
 
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner;
@@ -416,8 +417,8 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       // Update the salient file attributes.
       newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
       newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.isInECZone(iip));
+      updateBlocks(fsDir, addCloseOp, iip, newFile,
+          fsDir.getECSchema(iip), fsDir.isInECZone(iip));
       break;
       break;
     }
     }
     case OP_CLOSE: {
     case OP_CLOSE: {
@@ -437,8 +438,8 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       // Update the salient file attributes.
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
       file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, addCloseOp, iip, file, fsDir.isInECZone(iip));
+      updateBlocks(fsDir, addCloseOp, iip, file,
+          fsDir.getECSchema(iip), fsDir.isInECZone(iip));
 
 
       // Now close the file
       // Now close the file
       if (!file.isUnderConstruction() &&
       if (!file.isUnderConstruction() &&
@@ -496,9 +497,9 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // Update in-memory data structures
       // Update in-memory data structures
-      // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.isInECZone(iip));
-      
+      updateBlocks(fsDir, updateOp, iip, oldFile,
+          fsDir.getECSchema(iip), fsDir.isInECZone(iip));
+
       if (toAddRetryCache) {
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
       }
       }
@@ -514,8 +515,8 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // add the new block to the INodeFile
       // add the new block to the INodeFile
-      // TODO whether the file is striped should later be retrieved from iip
-      addNewBlock(addBlockOp, oldFile, fsDir.isInECZone(iip));
+      addNewBlock(addBlockOp, oldFile,
+          fsDir.getECSchema(iip), fsDir.isInECZone(iip));
       break;
       break;
     }
     }
     case OP_SET_REPLICATION: {
     case OP_SET_REPLICATION: {
@@ -956,7 +957,8 @@ public class FSEditLogLoader {
   /**
   /**
    * Add a new block into the given INodeFile
    * Add a new block into the given INodeFile
    */
    */
-  private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped)
+  private void addNewBlock(AddBlockOp op, INodeFile file,
+                           ECSchema schema, boolean isStriped)
       throws IOException {
       throws IOException {
     BlockInfo[] oldBlocks = file.getBlocks();
     BlockInfo[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
     Block pBlock = op.getPenultimateBlock();
@@ -985,8 +987,7 @@ public class FSEditLogLoader {
     // add the new block
     // add the new block
     final BlockInfo newBlockInfo;
     final BlockInfo newBlockInfo;
     if (isStriped) {
     if (isStriped) {
-      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
-          HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, schema);
     } else {
     } else {
       newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
       newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
           file.getPreferredBlockReplication());
           file.getPreferredBlockReplication());
@@ -1001,7 +1002,8 @@ public class FSEditLogLoader {
    * @throws IOException
    * @throws IOException
    */
    */
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
-      INodesInPath iip, INodeFile file, boolean isStriped) throws IOException {
+      INodesInPath iip, INodeFile file, ECSchema schema,
+      boolean isStriped) throws IOException {
     // Update its block list
     // Update its block list
     BlockInfo[] oldBlocks = file.getBlocks();
     BlockInfo[] oldBlocks = file.getBlocks();
     Block[] newBlocks = op.getBlocks();
     Block[] newBlocks = op.getBlocks();
@@ -1069,8 +1071,7 @@ public class FSEditLogLoader {
           // what about an old-version fsync() where fsync isn't called
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
           // until several blocks in?
           if (isStriped) {
           if (isStriped) {
-            newBI = new BlockInfoStripedUnderConstruction(newBlock,
-                HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+            newBI = new BlockInfoStripedUnderConstruction(newBlock, schema);
           } else {
           } else {
             newBI = new BlockInfoContiguousUnderConstruction(newBlock,
             newBI = new BlockInfoContiguousUnderConstruction(newBlock,
                 file.getPreferredBlockReplication());
                 file.getPreferredBlockReplication());
@@ -1080,8 +1081,9 @@ public class FSEditLogLoader {
           // is only executed when loading edits written by prior
           // is only executed when loading edits written by prior
           // versions of Hadoop. Current versions always log
           // versions of Hadoop. Current versions always log
           // OP_ADD operations as each block is allocated.
           // OP_ADD operations as each block is allocated.
+          // TODO: ECSchema can be restored from persisted file (HDFS-7859).
           newBI = isStriped ? new BlockInfoStriped(newBlock,
           newBI = isStriped ? new BlockInfoStriped(newBlock,
-              HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS) :
+              ErasureCodingSchemaManager.getSystemDefaultSchema()) :
               new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication());
               new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication());
         }
         }
         fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);
         fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);

+ 6 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.annotations.VisibleForTesting;
@@ -762,6 +763,9 @@ public class FSImageFormat {
             NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
             NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
             && (in.readBoolean());
             && (in.readBoolean());
     final int numBlocks = in.readInt();
     final int numBlocks = in.readInt();
+    // TODO: ECSchema can be restored from persisted file (HDFS-7859).
+    final ECSchema schema = isStriped ?
+        ErasureCodingSchemaManager.getSystemDefaultSchema() : null;
 
 
     if (numBlocks >= 0) {
     if (numBlocks >= 0) {
       // file
       // file
@@ -771,8 +775,7 @@ public class FSImageFormat {
       if (isStriped) {
       if (isStriped) {
         blocks = new Block[numBlocks];
         blocks = new Block[numBlocks];
         for (int j = 0; j < numBlocks; j++) {
         for (int j = 0; j < numBlocks; j++) {
-          blocks[j] = new BlockInfoStriped(new Block(),
-              HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+          blocks[j] = new BlockInfoStriped(new Block(), schema);
           blocks[j].readFields(in);
           blocks[j].readFields(in);
         }
         }
       } else {
       } else {
@@ -804,8 +807,7 @@ public class FSImageFormat {
                 BlockInfoStriped lastStripedBlk = (BlockInfoStriped) lastBlk;
                 BlockInfoStriped lastStripedBlk = (BlockInfoStriped) lastBlk;
                 blocks[blocks.length - 1]
                 blocks[blocks.length - 1]
                         = new BlockInfoStripedUnderConstruction(lastBlk,
                         = new BlockInfoStripedUnderConstruction(lastBlk,
-                                lastStripedBlk.getDataBlockNum(),
-                                lastStripedBlk.getParityBlockNum());
+                                lastStripedBlk.getSchema());
               } else {
               } else {
                 blocks[blocks.length - 1]
                 blocks[blocks.length - 1]
                         = new BlockInfoContiguousUnderConstruction(lastBlk,
                         = new BlockInfoContiguousUnderConstruction(lastBlk,

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -69,6 +69,7 @@ import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList;
 import com.google.protobuf.ByteString;
 import com.google.protobuf.ByteString;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public final class FSImageFormatPBINode {
 public final class FSImageFormatPBINode {
@@ -354,10 +355,12 @@ public final class FSImageFormatPBINode {
 
 
       FileWithStripedBlocksFeature stripeFeature = null;
       FileWithStripedBlocksFeature stripeFeature = null;
       if (f.hasStripedBlocks()) {
       if (f.hasStripedBlocks()) {
+        // TODO: HDFS-7859
+        ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
         StripedBlocksFeature sb = f.getStripedBlocks();
         StripedBlocksFeature sb = f.getStripedBlocks();
         stripeFeature = file.addStripedBlocksFeature();
         stripeFeature = file.addStripedBlocksFeature();
         for (StripedBlockProto sp : sb.getBlocksList()) {
         for (StripedBlockProto sp : sb.getBlocksList()) {
-          stripeFeature.addBlock(PBHelper.convert(sp));
+          stripeFeature.addBlock(PBHelper.convert(sp, schema));
         }
         }
       }
       }
 
 
@@ -372,7 +375,7 @@ public final class FSImageFormatPBINode {
           if (stripeFeature != null) {
           if (stripeFeature != null) {
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
             ucBlk = new BlockInfoStripedUnderConstruction(striped,
             ucBlk = new BlockInfoStripedUnderConstruction(striped,
-                striped.getDataBlockNum(), striped.getParityBlockNum());
+                striped.getSchema());
           } else {
           } else {
             ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
             ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
                 replication);
                 replication);

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.ShortWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.hadoop.io.WritableUtils;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.xml.sax.ContentHandler;
 import org.xml.sax.ContentHandler;
 import org.xml.sax.SAXException;
 import org.xml.sax.SAXException;
 
 
@@ -130,7 +131,11 @@ public class FSImageSerialization {
     final boolean isStriped = NameNodeLayoutVersion.supports(
     final boolean isStriped = NameNodeLayoutVersion.supports(
         NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
         NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
         && (in.readBoolean());
         && (in.readBoolean());
-  
+
+    // TODO: ECSchema can be restored from persisted file (HDFS-7859).
+    final ECSchema schema = isStriped ?
+        ErasureCodingSchemaManager.getSystemDefaultSchema() : null;
+
     int numBlocks = in.readInt();
     int numBlocks = in.readInt();
 
 
     final BlockInfoContiguous[] blocksContiguous;
     final BlockInfoContiguous[] blocksContiguous;
@@ -140,15 +145,12 @@ public class FSImageSerialization {
       blocksStriped = new BlockInfoStriped[numBlocks];
       blocksStriped = new BlockInfoStriped[numBlocks];
       int i = 0;
       int i = 0;
       for (; i < numBlocks - 1; i++) {
       for (; i < numBlocks - 1; i++) {
-        blocksStriped[i] = new BlockInfoStriped(new Block(),
-            HdfsConstants.NUM_DATA_BLOCKS,
-            HdfsConstants.NUM_PARITY_BLOCKS);
+        blocksStriped[i] = new BlockInfoStriped(new Block(), schema);
         blocksStriped[i].readFields(in);
         blocksStriped[i].readFields(in);
       }
       }
       if (numBlocks > 0) {
       if (numBlocks > 0) {
         blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(),
         blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(),
-            HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS,
-            BlockUCState.UNDER_CONSTRUCTION, null);
+            schema, BlockUCState.UNDER_CONSTRUCTION, null);
         blocksStriped[i].readFields(in);
         blocksStriped[i].readFields(in);
       }
       }
     } else {
     } else {

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -3672,7 +3672,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           fileINode.getPreferredBlockSize() * numBlocks;
           fileINode.getPreferredBlockSize() * numBlocks;
 
 
       final BlockInfoStriped striped = new BlockInfoStriped(commitBlock,
       final BlockInfoStriped striped = new BlockInfoStriped(commitBlock,
-          numDataUnits, numParityUnits);
+          ecSchema);
       final long actualBlockGroupSize = striped.spaceConsumed();
       final long actualBlockGroupSize = striped.spaceConsumed();
 
 
       diff = fullBlockGroupSize - actualBlockGroupSize;
       diff = fullBlockGroupSize - actualBlockGroupSize;

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
+import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 import org.mockito.internal.util.reflection.Whitebox;
@@ -42,8 +44,10 @@ public class TestBlockInfoStriped {
   private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
   private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
   private static final long BASE_ID = -1600;
   private static final long BASE_ID = -1600;
   private static final Block baseBlock = new Block(BASE_ID);
   private static final Block baseBlock = new Block(BASE_ID);
+  private static final ECSchema testSchema
+      = ErasureCodingSchemaManager.getSystemDefaultSchema();
   private final BlockInfoStriped info = new BlockInfoStriped(baseBlock,
   private final BlockInfoStriped info = new BlockInfoStriped(baseBlock,
-      NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
+      testSchema);
 
 
   private Block[] createReportedBlocks(int num) {
   private Block[] createReportedBlocks(int num) {
     Block[] blocks = new Block[num];
     Block[] blocks = new Block[num];
@@ -231,7 +235,7 @@ public class TestBlockInfoStriped {
     ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
     ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
     DataOutput out = new DataOutputStream(byteStream);
     DataOutput out = new DataOutputStream(byteStream);
     BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes,
     BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes,
-        generationStamp), NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS);
+        generationStamp), testSchema);
 
 
     try {
     try {
       blk.write(out);
       blk.write(out);

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
 import org.apache.log4j.Level;
 import org.junit.Test;
 import org.junit.Test;
@@ -72,6 +73,9 @@ public class TestFSEditLogLoader {
   private static final File TEST_DIR = PathUtils.getTestDir(TestFSEditLogLoader.class);
   private static final File TEST_DIR = PathUtils.getTestDir(TestFSEditLogLoader.class);
 
 
   private static final int NUM_DATA_NODES = 0;
   private static final int NUM_DATA_NODES = 0;
+
+  private static final ECSchema testSchema
+      = ErasureCodingSchemaManager.getSystemDefaultSchema();
   
   
   @Test
   @Test
   public void testDisplayRecentEditLogOpCodes() throws IOException {
   public void testDisplayRecentEditLogOpCodes() throws IOException {
@@ -459,7 +463,7 @@ public class TestFSEditLogLoader {
 
 
       // Add a striped block to the file
       // Add a striped block to the file
       BlockInfoStriped stripedBlk = new BlockInfoStriped(
       BlockInfoStriped stripedBlk = new BlockInfoStriped(
-          new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum);
+          new Block(blkId, blkNumBytes, timestamp), testSchema);
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       file.toUnderConstruction(clientName, clientMachine);
       file.toUnderConstruction(clientName, clientMachine);
       file.getStripedBlocksFeature().addBlock(stripedBlk);
       file.getStripedBlocksFeature().addBlock(stripedBlk);
@@ -525,7 +529,7 @@ public class TestFSEditLogLoader {
       Path p = new Path(testFilePath);
       Path p = new Path(testFilePath);
       DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
       DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
       BlockInfoStriped stripedBlk = new BlockInfoStriped(
       BlockInfoStriped stripedBlk = new BlockInfoStriped(
-          new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum);
+          new Block(blkId, blkNumBytes, timestamp), testSchema);
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       file.toUnderConstruction(clientName, clientMachine);
       file.toUnderConstruction(clientName, clientMachine);
       file.getStripedBlocksFeature().addBlock(stripedBlk);
       file.getStripedBlocksFeature().addBlock(stripedBlk);

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.junit.Assert;
 import org.junit.Assert;
 
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
@@ -67,6 +68,9 @@ public class TestFSImage {
 
 
   private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ =
   private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ =
       "image-with-zero-block-size.tar.gz";
       "image-with-zero-block-size.tar.gz";
+  private static final ECSchema testSchema
+      = ErasureCodingSchemaManager.getSystemDefaultSchema();
+
   @Test
   @Test
   public void testPersist() throws IOException {
   public void testPersist() throws IOException {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();
@@ -158,7 +162,7 @@ public class TestFSImage {
     for (int i = 0; i < stripedBlks.length; i++) {
     for (int i = 0; i < stripedBlks.length; i++) {
       stripedBlks[i] = new BlockInfoStriped(
       stripedBlks[i] = new BlockInfoStriped(
               new Block(stripedBlkId + i, preferredBlockSize, timestamp),
               new Block(stripedBlkId + i, preferredBlockSize, timestamp),
-              HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+              testSchema);
       file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
       file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
     }
     }
 
 

+ 12 - 27
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java

@@ -51,6 +51,9 @@ public class TestStripedINodeFile {
   private final BlockStoragePolicy defaultPolicy =
   private final BlockStoragePolicy defaultPolicy =
       defaultSuite.getDefaultPolicy();
       defaultSuite.getDefaultPolicy();
 
 
+  private static final ECSchema testSchema
+      = ErasureCodingSchemaManager.getSystemDefaultSchema();
+
   private static INodeFile createStripedINodeFile() {
   private static INodeFile createStripedINodeFile() {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
         null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID);
         null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID);
@@ -69,9 +72,7 @@ public class TestStripedINodeFile {
     ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk, testSchema);
     assertEquals(9, blockInfoStriped.getTotalBlockNum());
     assertEquals(9, blockInfoStriped.getTotalBlockNum());
   }
   }
 
 
@@ -83,9 +84,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk, testSchema);
     inf.addBlock(blockInfoStriped);
     inf.addBlock(blockInfoStriped);
     assertEquals(1, inf.getBlocks().length);
     assertEquals(1, inf.getBlocks().length);
   }
   }
@@ -98,9 +97,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk, testSchema);
     blockInfoStriped.setNumBytes(1);
     blockInfoStriped.setNumBytes(1);
     inf.addBlock(blockInfoStriped);
     inf.addBlock(blockInfoStriped);
     //   0. Calculate the total bytes per stripes <Num Bytes per Stripes>
     //   0. Calculate the total bytes per stripes <Num Bytes per Stripes>
@@ -127,15 +124,11 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     inf.addStripedBlocksFeature();
     Block blk1 = new Block(1);
     Block blk1 = new Block(1);
     BlockInfoStriped blockInfoStriped1
     BlockInfoStriped blockInfoStriped1
-        = new BlockInfoStriped(blk1,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk1, testSchema);
     blockInfoStriped1.setNumBytes(1);
     blockInfoStriped1.setNumBytes(1);
     Block blk2 = new Block(2);
     Block blk2 = new Block(2);
     BlockInfoStriped blockInfoStriped2
     BlockInfoStriped blockInfoStriped2
-        = new BlockInfoStriped(blk2,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk2, testSchema);
     blockInfoStriped2.setNumBytes(1);
     blockInfoStriped2.setNumBytes(1);
     inf.addBlock(blockInfoStriped1);
     inf.addBlock(blockInfoStriped1);
     inf.addBlock(blockInfoStriped2);
     inf.addBlock(blockInfoStriped2);
@@ -152,9 +145,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk, testSchema);
     blockInfoStriped.setNumBytes(100);
     blockInfoStriped.setNumBytes(100);
     inf.addBlock(blockInfoStriped);
     inf.addBlock(blockInfoStriped);
     // Compute file size should return actual data
     // Compute file size should return actual data
@@ -171,9 +162,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStripedUnderConstruction bInfoStripedUC
     BlockInfoStripedUnderConstruction bInfoStripedUC
-        = new BlockInfoStripedUnderConstruction(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStripedUnderConstruction(blk, testSchema);
     bInfoStripedUC.setNumBytes(100);
     bInfoStripedUC.setNumBytes(100);
     inf.addBlock(bInfoStripedUC);
     inf.addBlock(bInfoStripedUC);
     assertEquals(100, inf.computeFileSize());
     assertEquals(100, inf.computeFileSize());
@@ -188,9 +177,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStriped(blk, testSchema);
     blockInfoStriped.setNumBytes(100);
     blockInfoStriped.setNumBytes(100);
     inf.addBlock(blockInfoStriped);
     inf.addBlock(blockInfoStriped);
 
 
@@ -213,9 +200,7 @@ public class TestStripedINodeFile {
     inf.addStripedBlocksFeature();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStripedUnderConstruction bInfoStripedUC
     BlockInfoStripedUnderConstruction bInfoStripedUC
-        = new BlockInfoStripedUnderConstruction(blk,
-            (short)defaultSchema.getNumDataUnits(),
-            (short)defaultSchema.getNumParityUnits());
+        = new BlockInfoStripedUnderConstruction(blk, testSchema);
     bInfoStripedUC.setNumBytes(100);
     bInfoStripedUC.setNumBytes(100);
     inf.addBlock(bInfoStripedUC);
     inf.addBlock(bInfoStripedUC);