Explorar o código

HDFS-8580. Erasure coding: Persist cellSize in BlockInfoStriped and StripedBlocksFeature. Contributed by Walter Su.

Jing Zhao %!s(int64=10) %!d(string=hai) anos
pai
achega
c12a974ccf

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

@@ -302,3 +302,6 @@
 
     HDFS-8559. Erasure Coding: fix non-protobuf fsimage for striped blocks.
     (Jing Zhao via yliu)
+
+    HDFS-8580. Erasure coding: Persist cellSize in BlockInfoStriped and
+    StripedBlocksFeature. (Walter Su via jing9)

+ 9 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java

@@ -39,6 +39,7 @@ import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_S
  */
 public class BlockInfoStriped extends BlockInfo {
   private final ECSchema schema;
+  private final int cellSize;
   /**
    * 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
@@ -46,15 +47,16 @@ public class BlockInfoStriped extends BlockInfo {
    */
   private byte[] indices;
 
-  public BlockInfoStriped(Block blk, ECSchema schema) {
+  public BlockInfoStriped(Block blk, ECSchema schema, int cellSize) {
     super(blk, (short) (schema.getNumDataUnits() + schema.getNumParityUnits()));
     indices = new byte[schema.getNumDataUnits() + schema.getNumParityUnits()];
     initIndices();
     this.schema = schema;
+    this.cellSize = cellSize;
   }
 
   BlockInfoStriped(BlockInfoStriped b) {
-    this(b, b.getSchema());
+    this(b, b.getSchema(), b.getCellSize());
     this.setBlockCollection(b.getBlockCollection());
   }
 
@@ -75,6 +77,10 @@ public class BlockInfoStriped extends BlockInfo {
     return schema;
   }
 
+  public int getCellSize() {
+    return cellSize;
+  }
+
   private void initIndices() {
     for (int i = 0; i < indices.length; i++) {
       indices[i] = -1;
@@ -236,7 +242,7 @@ public class BlockInfoStriped extends BlockInfo {
       BlockUCState s, DatanodeStorageInfo[] targets) {
     final BlockInfoStripedUnderConstruction ucBlock;
     if(isComplete()) {
-      ucBlock = new BlockInfoStripedUnderConstruction(this, schema,
+      ucBlock = new BlockInfoStripedUnderConstruction(this, schema, cellSize,
           s, targets);
       ucBlock.setBlockCollection(getBlockCollection());
     } else {

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

@@ -57,16 +57,17 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
   /**
    * Constructor with null storage targets.
    */
-  public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema) {
-    this(blk, schema, UNDER_CONSTRUCTION, null);
+  public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema,
+      int cellSize) {
+    this(blk, schema, cellSize, UNDER_CONSTRUCTION, null);
   }
 
   /**
    * Create a striped block that is currently being constructed.
    */
   public BlockInfoStripedUnderConstruction(Block blk, ECSchema schema,
-      BlockUCState state, DatanodeStorageInfo[] targets) {
-    super(blk, schema);
+      int cellSize, BlockUCState state, DatanodeStorageInfo[] targets) {
+    super(blk, schema, cellSize);
     assert getBlockUCState() != COMPLETE :
       "BlockInfoStripedUnderConstruction cannot be in COMPLETE state";
     this.blockUCState = state;

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

@@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -531,8 +532,9 @@ class FSDirWriteFileOp {
       // associate new last block for the file
       final BlockInfo blockInfo;
       if (isStriped) {
-        ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema(
+        ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
             fsd.getFSNamesystem(), inodesInPath);
+        ECSchema ecSchema = ecZone.getSchema();
         short numDataUnits = (short) ecSchema.getNumDataUnits();
         short numParityUnits = (short) ecSchema.getNumParityUnits();
         short numLocations = (short) (numDataUnits + numParityUnits);
@@ -541,8 +543,8 @@ class FSDirWriteFileOp {
         fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
             numLocations, true);
         blockInfo = new BlockInfoStripedUnderConstruction(block, ecSchema,
-            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
-            targets);
+            ecZone.getCellSize(),
+            HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
       } else {
         // check quota limits and updated space consumed
         fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),

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

@@ -36,6 +36,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
@@ -417,9 +418,9 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema(
+      ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
           fsDir.getFSNamesystem(), iip);
-      updateBlocks(fsDir, addCloseOp, iip, newFile, ecSchema);
+      updateBlocks(fsDir, addCloseOp, iip, newFile, ecZone);
       break;
     }
     case OP_CLOSE: {
@@ -439,9 +440,9 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema(
+      ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
           fsDir.getFSNamesystem(), iip);
-      updateBlocks(fsDir, addCloseOp, iip, file, ecSchema);
+      updateBlocks(fsDir, addCloseOp, iip, file, ecZone);
 
       // Now close the file
       if (!file.isUnderConstruction() &&
@@ -499,9 +500,9 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // Update in-memory data structures
-      ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema(
+      ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
           fsDir.getFSNamesystem(), iip);
-      updateBlocks(fsDir, updateOp, iip, oldFile, ecSchema);
+      updateBlocks(fsDir, updateOp, iip, oldFile, ecZone);
 
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
@@ -518,9 +519,9 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // add the new block to the INodeFile
-      ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema(
+      ErasureCodingZone ecZone = FSDirErasureCodingOp.getErasureCodingZone(
           fsDir.getFSNamesystem(), iip);
-      addNewBlock(addBlockOp, oldFile, ecSchema);
+      addNewBlock(addBlockOp, oldFile, ecZone);
       break;
     }
     case OP_SET_REPLICATION: {
@@ -961,8 +962,8 @@ public class FSEditLogLoader {
   /**
    * Add a new block into the given INodeFile
    */
-  private void addNewBlock(AddBlockOp op, INodeFile file, ECSchema ecSchema)
-      throws IOException {
+  private void addNewBlock(AddBlockOp op, INodeFile file,
+      ErasureCodingZone ecZone) throws IOException {
     BlockInfo[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
     Block newBlock= op.getLastBlock();
@@ -989,9 +990,10 @@ public class FSEditLogLoader {
     }
     // add the new block
     final BlockInfo newBlockInfo;
-    boolean isStriped = ecSchema != null;
+    boolean isStriped = ecZone != null;
     if (isStriped) {
-      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock, ecSchema);
+      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
+          ecZone.getSchema(), ecZone.getCellSize());
     } else {
       newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
           file.getPreferredBlockReplication());
@@ -1006,7 +1008,8 @@ public class FSEditLogLoader {
    * @throws IOException
    */
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
-      INodesInPath iip, INodeFile file, ECSchema ecSchema) throws IOException {
+      INodesInPath iip, INodeFile file, ErasureCodingZone ecZone)
+      throws IOException {
     // Update its block list
     BlockInfo[] oldBlocks = file.getBlocks();
     Block[] newBlocks = op.getBlocks();
@@ -1065,7 +1068,7 @@ public class FSEditLogLoader {
         throw new IOException("Trying to delete non-existant block " + oldBlock);
       }
     } else if (newBlocks.length > oldBlocks.length) {
-      final boolean isStriped = ecSchema != null;
+      final boolean isStriped = ecZone != null;
       // We're adding blocks
       for (int i = oldBlocks.length; i < newBlocks.length; i++) {
         Block newBlock = newBlocks[i];
@@ -1075,7 +1078,8 @@ public class FSEditLogLoader {
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
           if (isStriped) {
-            newBI = new BlockInfoStripedUnderConstruction(newBlock, ecSchema);
+            newBI = new BlockInfoStripedUnderConstruction(newBlock,
+                ecZone.getSchema(), ecZone.getCellSize());
           } else {
             newBI = new BlockInfoContiguousUnderConstruction(newBlock,
                 file.getPreferredBlockReplication());
@@ -1086,9 +1090,14 @@ public class FSEditLogLoader {
           // versions of Hadoop. Current versions always log
           // OP_ADD operations as each block is allocated.
           // TODO: ECSchema can be restored from persisted file (HDFS-7859).
-          newBI = isStriped ? new BlockInfoStriped(newBlock,
-              ErasureCodingSchemaManager.getSystemDefaultSchema()) :
-              new BlockInfoContiguous(newBlock, file.getPreferredBlockReplication());
+          if (isStriped) {
+            newBI = new BlockInfoStriped(newBlock,
+                ErasureCodingSchemaManager.getSystemDefaultSchema(),
+                ecZone.getCellSize());
+          } else {
+            newBI = new BlockInfoContiguous(newBlock,
+                file.getPreferredBlockReplication());
+          }
         }
         fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file);
         file.addBlock(newBI);

+ 14 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -360,8 +360,13 @@ public final class FSImageFormatPBINode {
         // TODO: HDFS-7859
         ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
         stripeFeature = file.addStripedBlocksFeature();
-        for (BlockProto b : bp) {
-          stripeFeature.addBlock(new BlockInfoStriped(PBHelper.convert(b), schema));
+        if (bp.size() > 0) {
+          // if a striped file has block, the cellSize must exist in proto
+          final int cellSize = f.getStripedBlocks().getCellSize();
+          for (BlockProto b : bp) {
+            stripeFeature.addBlock(new BlockInfoStriped(PBHelper.convert(b),
+                schema, cellSize));
+          }
         }
       }
 
@@ -376,7 +381,7 @@ public final class FSImageFormatPBINode {
           if (stripeFeature != null) {
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
             ucBlk = new BlockInfoStripedUnderConstruction(striped,
-                striped.getSchema());
+                striped.getSchema(), striped.getCellSize());
           } else {
             ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
                 replication);
@@ -659,14 +664,17 @@ public final class FSImageFormatPBINode {
 
       FileWithStripedBlocksFeature sb = n.getStripedBlocksFeature();
       if (sb != null) {
+        StripedBlocksFeature.Builder builder =
+            StripedBlocksFeature.newBuilder();
         BlockInfoStriped[] sblocks = sb.getBlocks();
-        if (sblocks != null) {
+        if (sblocks != null && sblocks.length > 0) {
+          final int cellSize = sblocks[0].getCellSize();
           for (BlockInfoStriped sblk : sblocks) {
+            assert cellSize == sblk.getCellSize();
             b.addBlocks(PBHelper.convert(sblk));
           }
+          builder.setCellSize(cellSize);
         }
-        StripedBlocksFeature.Builder builder =
-            StripedBlocksFeature.newBuilder();
         b.setStripedBlocks(builder.build());
       }
 

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

@@ -3675,8 +3675,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final long diff;
     final short replicationFactor;
     if (fileINode.isStriped()) {
-      final ECSchema ecSchema = FSDirErasureCodingOp.getErasureCodingSchema(
-          this, iip);
+      final ErasureCodingZone ecZone = FSDirErasureCodingOp
+          .getErasureCodingZone(this, iip);
+      final ECSchema ecSchema = ecZone.getSchema();
       final short numDataUnits = (short) ecSchema.getNumDataUnits();
       final short numParityUnits = (short) ecSchema.getNumParityUnits();
 
@@ -3685,7 +3686,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           fileINode.getPreferredBlockSize() * numBlocks;
 
       final BlockInfoStriped striped = new BlockInfoStriped(commitBlock,
-          ecSchema);
+          ecSchema, ecZone.getCellSize());
       final long actualBlockGroupSize = striped.spaceConsumed();
 
       diff = fullBlockGroupSize - actualBlockGroupSize;

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto

@@ -94,6 +94,7 @@ message INodeSection {
 
   message StripedBlocksFeature {
     // store striped blocks related information
+    optional uint32 cellSize = 1;
   }
 
   message AclFeatureProto {

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

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult;
 import org.apache.hadoop.hdfs.server.namenode.ErasureCodingSchemaManager;
 import org.apache.hadoop.io.erasurecode.ECSchema;
@@ -46,8 +47,9 @@ public class TestBlockInfoStriped {
   private static final Block baseBlock = new Block(BASE_ID);
   private static final ECSchema testSchema
       = ErasureCodingSchemaManager.getSystemDefaultSchema();
+  private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
   private final BlockInfoStriped info = new BlockInfoStriped(baseBlock,
-      testSchema);
+      testSchema, cellSize);
 
   private Block[] createReportedBlocks(int num) {
     Block[] blocks = new Block[num];
@@ -235,7 +237,7 @@ public class TestBlockInfoStriped {
     ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
     DataOutput out = new DataOutputStream(byteStream);
     BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes,
-        generationStamp), testSchema);
+        generationStamp), testSchema, cellSize);
 
     try {
       blk.write(out);

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

@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -448,6 +449,7 @@ public class TestFSEditLogLoader {
       long timestamp = 1426222918;
       short blockNum = HdfsConstants.NUM_DATA_BLOCKS;
       short parityNum = HdfsConstants.NUM_PARITY_BLOCKS;
+      int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
@@ -463,7 +465,7 @@ public class TestFSEditLogLoader {
 
       // Add a striped block to the file
       BlockInfoStriped stripedBlk = new BlockInfoStriped(
-          new Block(blkId, blkNumBytes, timestamp), testSchema);
+          new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize);
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       file.toUnderConstruction(clientName, clientMachine);
       file.getStripedBlocksFeature().addBlock(stripedBlk);
@@ -488,6 +490,7 @@ public class TestFSEditLogLoader {
       assertEquals(timestamp, blks[0].getGenerationStamp());
       assertEquals(blockNum, blks[0].getDataBlockNum());
       assertEquals(parityNum, blks[0].getParityBlockNum());
+      assertEquals(cellSize, blks[0].getCellSize());
 
       cluster.shutdown();
       cluster = null;
@@ -520,6 +523,7 @@ public class TestFSEditLogLoader {
       long timestamp = 1426222918;
       short blockNum = HdfsConstants.NUM_DATA_BLOCKS;
       short parityNum = HdfsConstants.NUM_PARITY_BLOCKS;
+      int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
@@ -529,7 +533,7 @@ public class TestFSEditLogLoader {
       Path p = new Path(testFilePath);
       DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
       BlockInfoStriped stripedBlk = new BlockInfoStriped(
-          new Block(blkId, blkNumBytes, timestamp), testSchema);
+          new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize);
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       file.toUnderConstruction(clientName, clientMachine);
       file.getStripedBlocksFeature().addBlock(stripedBlk);
@@ -567,6 +571,7 @@ public class TestFSEditLogLoader {
       assertEquals(newTimestamp, blks[0].getGenerationStamp());
       assertEquals(blockNum, blks[0].getDataBlockNum());
       assertEquals(parityNum, blks[0].getParityBlockNum());
+      assertEquals(cellSize, blks[0].getCellSize());
 
       cluster.shutdown();
       cluster = null;

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

@@ -70,6 +70,7 @@ public class TestFSImage {
       "image-with-zero-block-size.tar.gz";
   private static final ECSchema testSchema
       = ErasureCodingSchemaManager.getSystemDefaultSchema();
+  private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 
   @Test
   public void testPersist() throws IOException {
@@ -162,7 +163,7 @@ public class TestFSImage {
     for (int i = 0; i < stripedBlks.length; i++) {
       stripedBlks[i] = new BlockInfoStriped(
               new Block(stripedBlkId + i, preferredBlockSize, timestamp),
-              testSchema);
+              testSchema, cellSize);
       file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
     }
 

+ 10 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java

@@ -53,6 +53,7 @@ public class TestStripedINodeFile {
 
   private static final ECSchema testSchema
       = ErasureCodingSchemaManager.getSystemDefaultSchema();
+  private static final int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 
   private static INodeFile createStripedINodeFile() {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
@@ -69,22 +70,20 @@ public class TestStripedINodeFile {
 
   @Test
   public void testBlockStripedTotalBlockCount() {
-    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk, testSchema);
+        = new BlockInfoStriped(blk, testSchema, cellSize);
     assertEquals(9, blockInfoStriped.getTotalBlockNum());
   }
 
   @Test
   public void testBlockStripedLength()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk, testSchema);
+        = new BlockInfoStriped(blk, testSchema, cellSize);
     inf.addBlock(blockInfoStriped);
     assertEquals(1, inf.getBlocks().length);
   }
@@ -92,12 +91,11 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedConsumedSpace()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk, testSchema);
+        = new BlockInfoStriped(blk, testSchema, cellSize);
     blockInfoStriped.setNumBytes(1);
     inf.addBlock(blockInfoStriped);
     //   0. Calculate the total bytes per stripes <Num Bytes per Stripes>
@@ -119,16 +117,15 @@ public class TestStripedINodeFile {
   @Test
   public void testMultipleBlockStripedConsumedSpace()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk1 = new Block(1);
     BlockInfoStriped blockInfoStriped1
-        = new BlockInfoStriped(blk1, testSchema);
+        = new BlockInfoStriped(blk1, testSchema, cellSize);
     blockInfoStriped1.setNumBytes(1);
     Block blk2 = new Block(2);
     BlockInfoStriped blockInfoStriped2
-        = new BlockInfoStriped(blk2, testSchema);
+        = new BlockInfoStriped(blk2, testSchema, cellSize);
     blockInfoStriped2.setNumBytes(1);
     inf.addBlock(blockInfoStriped1);
     inf.addBlock(blockInfoStriped2);
@@ -140,12 +137,11 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedFileSize()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk, testSchema);
+        = new BlockInfoStriped(blk, testSchema, cellSize);
     blockInfoStriped.setNumBytes(100);
     inf.addBlock(blockInfoStriped);
     // Compute file size should return actual data
@@ -157,12 +153,11 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedUCFileSize()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStripedUnderConstruction bInfoStripedUC
-        = new BlockInfoStripedUnderConstruction(blk, testSchema);
+        = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize);
     bInfoStripedUC.setNumBytes(100);
     inf.addBlock(bInfoStripedUC);
     assertEquals(100, inf.computeFileSize());
@@ -172,12 +167,11 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedComputeQuotaUsage()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
-        = new BlockInfoStriped(blk, testSchema);
+        = new BlockInfoStriped(blk, testSchema, cellSize);
     blockInfoStriped.setNumBytes(100);
     inf.addBlock(blockInfoStriped);
 
@@ -195,12 +189,11 @@ public class TestStripedINodeFile {
   @Test
   public void testBlockStripedUCComputeQuotaUsage()
       throws IOException, InterruptedException {
-    ECSchema defaultSchema = ErasureCodingSchemaManager.getSystemDefaultSchema();
     INodeFile inf = createStripedINodeFile();
     inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStripedUnderConstruction bInfoStripedUC
-        = new BlockInfoStripedUnderConstruction(blk, testSchema);
+        = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize);
     bInfoStripedUC.setNumBytes(100);
     inf.addBlock(bInfoStripedUC);