Browse Source

HDFS-8058. Erasure coding: use BlockInfo[] for both striped and contiguous blocks in INodeFile. Contributed by Zhe Zhang and Yi Liu.

Zhe Zhang 10 năm trước cách đây
mục cha
commit
7e091de136
25 tập tin đã thay đổi với 273 bổ sung494 xóa
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
  2. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  3. 0 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
  4. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  5. 3 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
  6. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
  7. 28 31
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
  8. 1 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  9. 32 44
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  10. 0 116
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java
  11. 140 227
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  12. 10 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
  13. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
  14. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java
  15. 9 9
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
  16. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileWithSnapshotFeature.java
  17. 2 6
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
  18. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  19. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
  20. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  21. 12 10
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
  22. 7 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  24. 3 11
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
  25. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTruncateQuotaUpdate.java

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

@@ -356,3 +356,6 @@
 
 
     HDFS-8619. Erasure Coding: revisit replica counting for striped blocks.
     HDFS-8619. Erasure Coding: revisit replica counting for striped blocks.
     (Jing Zhao via yliu)
     (Jing Zhao via yliu)
+
+    HDFS-8058. Erasure coding: use BlockInfo[] for both striped and contiguous
+    blocks in INodeFile. (Zhe Zhang and Yi Liu via zhz)

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.util.LightWeightGSet;
  */
  */
 public abstract class BlockInfo extends Block
 public abstract class BlockInfo extends Block
     implements LightWeightGSet.LinkedElement {
     implements LightWeightGSet.LinkedElement {
+  public static final BlockInfo[] EMPTY_ARRAY = {};
   private BlockCollection bc;
   private BlockCollection bc;
 
 
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */
   /** For implementing {@link LightWeightGSet.LinkedElement} interface */

+ 0 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java

@@ -26,7 +26,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class BlockInfoContiguous extends BlockInfo {
 public class BlockInfoContiguous extends BlockInfo {
-  public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
 
 
   public BlockInfoContiguous(short size) {
   public BlockInfoContiguous(short size) {
     super(size);
     super(size);

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

@@ -3037,13 +3037,13 @@ public class BlockManager {
   
   
   /** Set replication for the blocks. */
   /** Set replication for the blocks. */
   public void setReplication(final short oldRepl, final short newRepl,
   public void setReplication(final short oldRepl, final short newRepl,
-      final String src, final BlockInfoContiguous... blocks) {
+      final String src, final BlockInfo... blocks) {
     if (newRepl == oldRepl) {
     if (newRepl == oldRepl) {
       return;
       return;
     }
     }
 
 
     // update needReplication priority queues
     // update needReplication priority queues
-    for(BlockInfoContiguous b : blocks) {
+    for(BlockInfo b : blocks) {
       updateNeededReplications(b, 0, newRepl-oldRepl);
       updateNeededReplications(b, 0, newRepl-oldRepl);
     }
     }
       
       
@@ -3051,7 +3051,7 @@ public class BlockManager {
       // old replication > the new one; need to remove copies
       // old replication > the new one; need to remove copies
       LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl
       LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl
           + " for " + src);
           + " for " + src);
-      for(BlockInfoContiguous b : blocks) {
+      for(BlockInfo b : blocks) {
         processOverReplicatedBlock(b, newRepl, null, null);
         processOverReplicatedBlock(b, newRepl, null, null);
       }
       }
     } else { // replication factor is increased
     } else { // replication factor is increased

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

@@ -25,14 +25,12 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 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.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.util.EnumCounters;
 import org.apache.hadoop.hdfs.util.EnumCounters;
@@ -150,7 +148,7 @@ public class FSDirAttrOp {
       }
       }
 
 
       final short[] blockRepls = new short[2]; // 0: old, 1: new
       final short[] blockRepls = new short[2]; // 0: old, 1: new
-      final BlockInfoContiguous[] blocks = unprotectedSetReplication(fsd, src,
+      final BlockInfo[] blocks = unprotectedSetReplication(fsd, src,
           replication, blockRepls);
           replication, blockRepls);
       isFile = blocks != null;
       isFile = blocks != null;
       if (isFile) {
       if (isFile) {
@@ -377,7 +375,7 @@ public class FSDirAttrOp {
     }
     }
   }
   }
 
 
-  static BlockInfoContiguous[] unprotectedSetReplication(
+  static BlockInfo[] unprotectedSetReplication(
       FSDirectory fsd, String src, short replication, short[] blockRepls)
       FSDirectory fsd, String src, short replication, short[] blockRepls)
       throws QuotaExceededException, UnresolvedLinkException,
       throws QuotaExceededException, UnresolvedLinkException,
       SnapshotAccessControlException, UnsupportedActionException {
       SnapshotAccessControlException, UnsupportedActionException {
@@ -417,7 +415,7 @@ public class FSDirAttrOp {
       blockRepls[0] = oldBR;
       blockRepls[0] = oldBR;
       blockRepls[1] = newBR;
       blockRepls[1] = newBR;
     }
     }
-    return file.getContiguousBlocks();
+    return file.getBlocks();
   }
   }
 
 
   static void unprotectedSetStoragePolicy(
   static void unprotectedSetStoragePolicy(

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

@@ -234,7 +234,7 @@ class FSDirConcatOp {
     int count = 0;
     int count = 0;
     for (INodeFile nodeToRemove : srcList) {
     for (INodeFile nodeToRemove : srcList) {
       if(nodeToRemove != null) {
       if(nodeToRemove != null) {
-        nodeToRemove.setContiguousBlocks(null);
+        nodeToRemove.clearBlocks();
         nodeToRemove.getParent().removeChild(nodeToRemove);
         nodeToRemove.getParent().removeChild(nodeToRemove);
         fsd.getINodeMap().remove(nodeToRemove);
         fsd.getINodeMap().remove(nodeToRemove);
         count++;
         count++;

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

@@ -44,7 +44,6 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 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.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
@@ -481,25 +480,24 @@ class FSDirWriteFileOp {
       long preferredBlockSize, boolean underConstruction, String clientName,
       long preferredBlockSize, boolean underConstruction, String clientName,
       String clientMachine, byte storagePolicyId) {
       String clientMachine, byte storagePolicyId) {
     final INodeFile newNode;
     final INodeFile newNode;
+    Preconditions.checkNotNull(existing);
     assert fsd.hasWriteLock();
     assert fsd.hasWriteLock();
-    if (underConstruction) {
-      newNode = newINodeFile(id, permissions, modificationTime,
-          modificationTime, replication, preferredBlockSize, storagePolicyId);
-      newNode.toUnderConstruction(clientName, clientMachine);
-    } else {
-      newNode = newINodeFile(id, permissions, modificationTime, atime,
-          replication, preferredBlockSize, storagePolicyId);
-    }
-
-    newNode.setLocalName(localName);
     try {
     try {
+      // check if the file is in an EC zone
+      final boolean isStriped = FSDirErasureCodingOp.isInErasureCodingZone(
+          fsd.getFSNamesystem(), existing);
+      if (underConstruction) {
+        newNode = newINodeFile(id, permissions, modificationTime,
+            modificationTime, replication, preferredBlockSize, storagePolicyId,
+            isStriped);
+        newNode.toUnderConstruction(clientName, clientMachine);
+      } else {
+        newNode = newINodeFile(id, permissions, modificationTime, atime,
+            replication, preferredBlockSize, storagePolicyId, isStriped);
+      }
+      newNode.setLocalName(localName);
       INodesInPath iip = fsd.addINode(existing, newNode);
       INodesInPath iip = fsd.addINode(existing, newNode);
       if (iip != null) {
       if (iip != null) {
-        // check if the file is in an EC zone
-        if (FSDirErasureCodingOp.isInErasureCodingZone(fsd.getFSNamesystem(),
-            iip)) {
-          newNode.addStripedBlocksFeature();
-        }
         if (aclEntries != null) {
         if (aclEntries != null) {
           AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID);
           AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID);
         }
         }
@@ -580,21 +578,18 @@ class FSDirWriteFileOp {
       String clientName, String clientMachine)
       String clientName, String clientMachine)
       throws IOException {
       throws IOException {
 
 
+    Preconditions.checkNotNull(existing);
     long modTime = now();
     long modTime = now();
-    INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
-        modTime, modTime, replication, preferredBlockSize);
-    newNode.setLocalName(localName.getBytes(Charsets.UTF_8));
-    newNode.toUnderConstruction(clientName, clientMachine);
-
     INodesInPath newiip;
     INodesInPath newiip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
+      final boolean isStriped = FSDirErasureCodingOp.isInErasureCodingZone(
+          fsd.getFSNamesystem(), existing);
+      INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
+          modTime, modTime, replication, preferredBlockSize, isStriped);
+      newNode.setLocalName(localName.getBytes(Charsets.UTF_8));
+      newNode.toUnderConstruction(clientName, clientMachine);
       newiip = fsd.addINode(existing, newNode);
       newiip = fsd.addINode(existing, newNode);
-      if (newiip != null
-          && FSDirErasureCodingOp.isInErasureCodingZone(fsd.getFSNamesystem(),
-              newiip)) {
-        newNode.addStripedBlocksFeature();
-      }
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
@@ -788,16 +783,18 @@ class FSDirWriteFileOp {
 
 
   private static INodeFile newINodeFile(
   private static INodeFile newINodeFile(
       long id, PermissionStatus permissions, long mtime, long atime,
       long id, PermissionStatus permissions, long mtime, long atime,
-      short replication, long preferredBlockSize, byte storagePolicyId) {
+      short replication, long preferredBlockSize, byte storagePolicyId,
+      boolean isStriped) {
     return new INodeFile(id, null, permissions, mtime, atime,
     return new INodeFile(id, null, permissions, mtime, atime,
-        BlockInfoContiguous.EMPTY_ARRAY, replication, preferredBlockSize,
-        storagePolicyId);
+        BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
+        storagePolicyId, isStriped);
   }
   }
 
 
   private static INodeFile newINodeFile(long id, PermissionStatus permissions,
   private static INodeFile newINodeFile(long id, PermissionStatus permissions,
-      long mtime, long atime, short replication, long preferredBlockSize) {
+      long mtime, long atime, short replication, long preferredBlockSize,
+      boolean isStriped) {
     return newINodeFile(id, permissions, mtime, atime, replication,
     return newINodeFile(id, permissions, mtime, atime, replication,
-        preferredBlockSize, (byte)0);
+        preferredBlockSize, (byte)0, isStriped);
   }
   }
 
 
   /**
   /**

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

@@ -55,9 +55,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 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.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 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.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -76,7 +74,6 @@ 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;
@@ -903,7 +900,7 @@ public class FSImageFormat {
       final long preferredBlockSize = in.readLong();
       final long preferredBlockSize = in.readLong();
 
 
       return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
       return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
-          accessTime, replication, preferredBlockSize, (byte) 0, null);
+          accessTime, replication, preferredBlockSize, (byte) 0, null, false);
     }
     }
 
 
     public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)
     public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)

+ 32 - 44
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java

@@ -55,7 +55,6 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructio
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto;
-import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.StripedBlocksFeature;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrCompactProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrCompactProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.QuotaByStorageTypeEntryProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.QuotaByStorageTypeEntryProto;
@@ -326,13 +325,22 @@ public final class FSImageFormatPBINode {
       INodeSection.INodeFile f = n.getFile();
       INodeSection.INodeFile f = n.getFile();
       List<BlockProto> bp = f.getBlocksList();
       List<BlockProto> bp = f.getBlocksList();
       short replication = (short) f.getReplication();
       short replication = (short) f.getReplication();
+      boolean isStriped = f.getIsStriped();
       LoaderContext state = parent.getLoaderContext();
       LoaderContext state = parent.getLoaderContext();
+      ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
 
 
-      BlockInfoContiguous[] blocks = null;
-      if (!f.hasStripedBlocks()) {
-        blocks = new BlockInfoContiguous[bp.size()];
-        for (int i = 0, e = bp.size(); i < e; ++i) {
-          blocks[i] = new BlockInfoContiguous(PBHelper.convert(bp.get(i)), replication);
+      if (isStriped) {
+        Preconditions.checkState(f.hasStripingCellSize());
+      }
+      BlockInfo[] blocks = new BlockInfo[bp.size()];
+      for (int i = 0; i < bp.size(); ++i) {
+        BlockProto b = bp.get(i);
+        if (isStriped) {
+          blocks[i] = new BlockInfoStriped(PBHelper.convert(b), schema,
+              (int)f.getStripingCellSize());
+        } else {
+          blocks[i] = new BlockInfoContiguous(PBHelper.convert(b),
+              replication);
         }
         }
       }
       }
 
 
@@ -342,46 +350,31 @@ public final class FSImageFormatPBINode {
       final INodeFile file = new INodeFile(n.getId(),
       final INodeFile file = new INodeFile(n.getId(),
           n.getName().toByteArray(), permissions, f.getModificationTime(),
           n.getName().toByteArray(), permissions, f.getModificationTime(),
           f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
           f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
-          (byte)f.getStoragePolicyID());
+          (byte)f.getStoragePolicyID(), isStriped);
 
 
       if (f.hasAcl()) {
       if (f.hasAcl()) {
         int[] entries = AclEntryStatusFormat.toInt(loadAclEntries(
         int[] entries = AclEntryStatusFormat.toInt(loadAclEntries(
             f.getAcl(), state.getStringTable()));
             f.getAcl(), state.getStringTable()));
         file.addAclFeature(new AclFeature(entries));
         file.addAclFeature(new AclFeature(entries));
       }
       }
-      
+
       if (f.hasXAttrs()) {
       if (f.hasXAttrs()) {
         file.addXAttrFeature(new XAttrFeature(
         file.addXAttrFeature(new XAttrFeature(
             loadXAttrs(f.getXAttrs(), state.getStringTable())));
             loadXAttrs(f.getXAttrs(), state.getStringTable())));
       }
       }
 
 
-      FileWithStripedBlocksFeature stripeFeature = null;
-      if (f.hasStripedBlocks()) {
-        // TODO: HDFS-7859
-        ECSchema schema = ErasureCodingSchemaManager.getSystemDefaultSchema();
-        stripeFeature = file.addStripedBlocksFeature();
-        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));
-          }
-        }
-      }
-
       // under-construction information
       // under-construction information
       if (f.hasFileUC()) {
       if (f.hasFileUC()) {
         INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
         INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
         file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
-        BlockInfo lastBlk = file.getLastBlock();
-        if (lastBlk != null) {
+        if (blocks.length > 0) {
+          BlockInfo lastBlk = file.getLastBlock();
           // replace the last block of file
           // replace the last block of file
           final BlockInfo ucBlk;
           final BlockInfo ucBlk;
-          if (stripeFeature != null) {
+          if (isStriped) {
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
             ucBlk = new BlockInfoStripedUnderConstruction(striped,
             ucBlk = new BlockInfoStripedUnderConstruction(striped,
-                striped.getSchema(), striped.getCellSize());
+                schema, (int)f.getStripingCellSize());
           } else {
           } else {
             ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
             ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
                 replication);
                 replication);
@@ -500,7 +493,8 @@ public final class FSImageFormatPBINode {
           .setPermission(buildPermissionStatus(file, state.getStringMap()))
           .setPermission(buildPermissionStatus(file, state.getStringMap()))
           .setPreferredBlockSize(file.getPreferredBlockSize())
           .setPreferredBlockSize(file.getPreferredBlockSize())
           .setReplication(file.getFileReplication())
           .setReplication(file.getFileReplication())
-          .setStoragePolicyID(file.getLocalStoragePolicyID());
+          .setStoragePolicyID(file.getLocalStoragePolicyID())
+          .setIsStriped(file.isStriped());
 
 
       AclFeature f = file.getAclFeature();
       AclFeature f = file.getAclFeature();
       if (f != null) {
       if (f != null) {
@@ -654,28 +648,22 @@ public final class FSImageFormatPBINode {
     private void save(OutputStream out, INodeFile n) throws IOException {
     private void save(OutputStream out, INodeFile n) throws IOException {
       INodeSection.INodeFile.Builder b = buildINodeFile(n,
       INodeSection.INodeFile.Builder b = buildINodeFile(n,
           parent.getSaverContext());
           parent.getSaverContext());
+      BlockInfo[] blocks = n.getBlocks();
 
 
-      BlockInfoContiguous[] cBlks = n.getContiguousBlocks();
-      if (cBlks != null) {
-        for (Block block : cBlks) {
+      if (blocks != null) {
+        for (Block block : n.getBlocks()) {
           b.addBlocks(PBHelper.convert(block));
           b.addBlocks(PBHelper.convert(block));
         }
         }
       }
       }
 
 
-      FileWithStripedBlocksFeature sb = n.getStripedBlocksFeature();
-      if (sb != null) {
-        StripedBlocksFeature.Builder builder =
-            StripedBlocksFeature.newBuilder();
-        BlockInfoStriped[] sblocks = sb.getBlocks();
-        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);
+      if (n.isStriped()) {
+        if (blocks != null && blocks.length > 0) {
+          BlockInfo firstBlock = blocks[0];
+          Preconditions.checkState(firstBlock.isStriped());
+          b.setStripingCellSize(((BlockInfoStriped)firstBlock).getCellSize());
+        } else {
+          b.setStripingCellSize(HdfsConstants.BLOCK_STRIPED_CELL_SIZE);
         }
         }
-        b.setStripedBlocks(builder.build());
       }
       }
 
 
       FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature();
       FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature();

+ 0 - 116
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java

@@ -1,116 +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 com.google.common.base.Preconditions;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
-
-/**
- * Feature for file with striped blocks
- */
-class FileWithStripedBlocksFeature implements INode.Feature {
-  private BlockInfoStriped[] blocks;
-
-  FileWithStripedBlocksFeature() {
-    blocks = new BlockInfoStriped[0];
-  }
-
-  FileWithStripedBlocksFeature(BlockInfoStriped[] blocks) {
-    Preconditions.checkArgument(blocks != null);
-    this.blocks = blocks;
-  }
-
-  BlockInfoStriped[] getBlocks() {
-    return this.blocks;
-  }
-
-  void setBlock(int index, BlockInfoStriped blk) {
-    blocks[index] = blk;
-  }
-
-  BlockInfoStriped getLastBlock() {
-    return blocks == null || blocks.length == 0 ?
-        null : blocks[blocks.length - 1];
-  }
-
-  int numBlocks() {
-    return blocks == null ? 0 : blocks.length;
-  }
-
-  void updateBlockCollection(INodeFile file) {
-    if (blocks != null) {
-      for (BlockInfoStriped blk : blocks) {
-        blk.setBlockCollection(file);
-      }
-    }
-  }
-
-  private void setBlocks(BlockInfoStriped[] blocks) {
-    this.blocks = blocks;
-  }
-
-  void addBlock(BlockInfoStriped newBlock) {
-    if (this.blocks == null) {
-      this.setBlocks(new BlockInfoStriped[]{newBlock});
-    } else {
-      int size = this.blocks.length;
-      BlockInfoStriped[] newlist = new BlockInfoStriped[size + 1];
-      System.arraycopy(this.blocks, 0, newlist, 0, size);
-      newlist[size] = newBlock;
-      this.setBlocks(newlist);
-    }
-  }
-
-  BlockInfoStripedUnderConstruction removeLastBlock(
-      Block oldblock) {
-    if (blocks == null || blocks.length == 0) {
-      return null;
-    }
-    int newSize = blocks.length - 1;
-    if (!blocks[newSize].equals(oldblock)) {
-      return null;
-    }
-
-    BlockInfoStripedUnderConstruction uc =
-        (BlockInfoStripedUnderConstruction) blocks[newSize];
-    //copy to a new list
-    BlockInfoStriped[] newlist = new BlockInfoStriped[newSize];
-    System.arraycopy(blocks, 0, newlist, 0, newSize);
-    setBlocks(newlist);
-    return uc;
-  }
-
-  void truncateStripedBlocks(int n) {
-    final BlockInfoStriped[] newBlocks;
-    if (n == 0) {
-      newBlocks = new BlockInfoStriped[0];
-    } else {
-      newBlocks = new BlockInfoStriped[n];
-      System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
-    }
-    // set new blocks
-    setBlocks(newBlocks);
-  }
-
-  void clear() {
-    this.blocks = null;
-  }
-}

+ 140 - 227
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -40,9 +40,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 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.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -84,12 +82,14 @@ public class INodeFile extends INodeWithAdditionalFields
 
 
   /** 
   /** 
    * Bit format:
    * Bit format:
-   * [4-bit storagePolicyID][12-bit replication][48-bit preferredBlockSize]
+   * [4-bit storagePolicyID][1-bit isStriped]
+   * [11-bit replication][48-bit preferredBlockSize]
    */
    */
   static enum HeaderFormat {
   static enum HeaderFormat {
     PREFERRED_BLOCK_SIZE(null, 48, 1),
     PREFERRED_BLOCK_SIZE(null, 48, 1),
-    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 0),
-    STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
+    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 11, 0),
+    IS_STRIPED(REPLICATION.BITS, 1, 0),
+    STORAGE_POLICY_ID(IS_STRIPED.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
         0);
         0);
 
 
     private final LongBitFormat BITS;
     private final LongBitFormat BITS;
@@ -110,14 +110,27 @@ public class INodeFile extends INodeWithAdditionalFields
       return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
       return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
     }
     }
 
 
+    static boolean isStriped(long header) {
+      long isStriped = IS_STRIPED.BITS.retrieve(header);
+      Preconditions.checkState(isStriped == 0 || isStriped == 1);
+      return isStriped == 1;
+    }
+
     static long toLong(long preferredBlockSize, short replication,
     static long toLong(long preferredBlockSize, short replication,
-        byte storagePolicyID) {
+        boolean isStriped, byte storagePolicyID) {
       long h = 0;
       long h = 0;
       if (preferredBlockSize == 0) {
       if (preferredBlockSize == 0) {
         preferredBlockSize = PREFERRED_BLOCK_SIZE.BITS.getMin();
         preferredBlockSize = PREFERRED_BLOCK_SIZE.BITS.getMin();
       }
       }
       h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h);
       h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h);
-      h = REPLICATION.BITS.combine(replication, h);
+      // Replication factor for striped files is zero
+      if (isStriped) {
+        h = REPLICATION.BITS.combine(0L, h);
+        h = IS_STRIPED.BITS.combine(1L, h);
+      } else {
+        h = REPLICATION.BITS.combine(replication, h);
+        h = IS_STRIPED.BITS.combine(0L, h);
+      }
       h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h);
       h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h);
       return h;
       return h;
     }
     }
@@ -126,21 +139,27 @@ public class INodeFile extends INodeWithAdditionalFields
 
 
   private long header = 0L;
   private long header = 0L;
 
 
-  private BlockInfoContiguous[] blocks;
+  private BlockInfo[] blocks;
 
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
-            long atime, BlockInfoContiguous[] blklist, short replication,
+            long atime, BlockInfo[] blklist, short replication,
             long preferredBlockSize) {
             long preferredBlockSize) {
     this(id, name, permissions, mtime, atime, blklist, replication,
     this(id, name, permissions, mtime, atime, blklist, replication,
-         preferredBlockSize, (byte) 0);
+        preferredBlockSize, (byte) 0, false);
   }
   }
 
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
-      long atime, BlockInfoContiguous[] blklist, short replication,
-      long preferredBlockSize, byte storagePolicyID) {
+      long atime, BlockInfo[] blklist, short replication,
+      long preferredBlockSize, byte storagePolicyID, boolean isStriped) {
     super(id, name, permissions, mtime, atime);
     super(id, name, permissions, mtime, atime);
-    header = HeaderFormat.toLong(preferredBlockSize, replication, storagePolicyID);
-    this.blocks = blklist;
+    header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
+        storagePolicyID);
+    if (blklist != null && blklist.length > 0) {
+      for (BlockInfo b : blklist) {
+        Preconditions.checkArgument(b.isStriped() == isStriped);
+      }
+    }
+    setBlocks(blklist);
   }
   }
   
   
   public INodeFile(INodeFile that) {
   public INodeFile(INodeFile that) {
@@ -177,27 +196,6 @@ public class INodeFile extends INodeWithAdditionalFields
         && getXAttrFeature() == other.getXAttrFeature();
         && getXAttrFeature() == other.getXAttrFeature();
   }
   }
 
 
-  /* Start of StripedBlock Feature */
-
-  public final FileWithStripedBlocksFeature getStripedBlocksFeature() {
-    return getFeature(FileWithStripedBlocksFeature.class);
-  }
-
-  public FileWithStripedBlocksFeature addStripedBlocksFeature() {
-    assert blocks == null || blocks.length == 0:
-        "The file contains contiguous blocks";
-    assert !isStriped();
-    this.setFileReplication((short) 0);
-    FileWithStripedBlocksFeature sb = new FileWithStripedBlocksFeature();
-    addFeature(sb);
-    return sb;
-  }
-
-  /** Used to make sure there is no contiguous block related info */
-  private boolean hasNoContiguousBlock() {
-    return (blocks == null || blocks.length == 0) && getFileReplication() == 0;
-  }
-
   /* Start of Under-Construction Feature */
   /* Start of Under-Construction Feature */
 
 
   /**
   /**
@@ -232,7 +230,7 @@ public class INodeFile extends INodeWithAdditionalFields
         "file is no longer under construction");
         "file is no longer under construction");
     FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
     FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
     if (uc != null) {
     if (uc != null) {
-      assertAllBlocksComplete(getBlocks());
+      assertAllBlocksComplete();
       removeFeature(uc);
       removeFeature(uc);
       this.setModificationTime(mtime);
       this.setModificationTime(mtime);
     }
     }
@@ -240,32 +238,21 @@ public class INodeFile extends INodeWithAdditionalFields
   }
   }
 
 
   /** Assert all blocks are complete. */
   /** Assert all blocks are complete. */
-  private void assertAllBlocksComplete(BlockInfo[] blks) {
-    if (blks == null) {
+  private void assertAllBlocksComplete() {
+    if (blocks == null) {
       return;
       return;
     }
     }
-    for (int i = 0; i < blks.length; i++) {
-      Preconditions.checkState(blks[i].isComplete(), "Failed to finalize"
+    for (int i = 0; i < blocks.length; i++) {
+      Preconditions.checkState(blocks[i].isComplete(), "Failed to finalize"
           + " %s %s since blocks[%s] is non-complete, where blocks=%s.",
           + " %s %s since blocks[%s] is non-complete, where blocks=%s.",
-          getClass().getSimpleName(), this, i, Arrays.asList(blks));
+          getClass().getSimpleName(), this, i, Arrays.asList(blocks));
     }
     }
   }
   }
 
 
-  /**
-   * Instead of adding a new block, this function is usually used while loading
-   * fsimage or converting the last block to UC/Complete.
-   */
   @Override // BlockCollection
   @Override // BlockCollection
   public void setBlock(int index, BlockInfo blk) {
   public void setBlock(int index, BlockInfo blk) {
-    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
-    if (sb == null) {
-      assert !blk.isStriped();
-      this.blocks[index] = (BlockInfoContiguous) blk;
-    } else {
-      assert blk.isStriped();
-      assert hasNoContiguousBlock();
-      sb.setBlock(index, (BlockInfoStriped) blk);
-    }
+    Preconditions.checkArgument(blk.isStriped() == this.isStriped());
+    this.blocks[index] = blk;
   }
   }
 
 
   @Override // BlockCollection, the file should be under construction
   @Override // BlockCollection, the file should be under construction
@@ -278,15 +265,13 @@ public class INodeFile extends INodeWithAdditionalFields
     }
     }
 
 
     final BlockInfo ucBlock;
     final BlockInfo ucBlock;
-    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
-    if (sb == null) {
-      assert !lastBlock.isStriped();
-      ucBlock = ((BlockInfoContiguous) lastBlock)
+    if (isStriped()) {
+      Preconditions.checkState(lastBlock.isStriped());
+      ucBlock = ((BlockInfoStriped) lastBlock)
           .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
           .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
     } else {
     } else {
-      assert hasNoContiguousBlock();
-      assert lastBlock.isStriped();
-      ucBlock = ((BlockInfoStriped) lastBlock)
+      Preconditions.checkState(!lastBlock.isStriped());
+      ucBlock = ((BlockInfoContiguous) lastBlock)
           .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
           .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
     }
     }
     setBlock(numBlocks() - 1, ucBlock);
     setBlock(numBlocks() - 1, ucBlock);
@@ -299,27 +284,21 @@ public class INodeFile extends INodeWithAdditionalFields
   BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
   BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
     Preconditions.checkState(isUnderConstruction(),
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
         "file is no longer under construction");
-    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
-    if (sb == null) {
-      if (blocks == null || blocks.length == 0) {
-        return null;
-      }
-      int size_1 = blocks.length - 1;
-      if (!blocks[size_1].equals(oldblock)) {
-        return null;
-      }
-
-      BlockInfoContiguousUnderConstruction uc =
-          (BlockInfoContiguousUnderConstruction)blocks[size_1];
-      //copy to a new list
-      BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1];
-      System.arraycopy(blocks, 0, newlist, 0, size_1);
-      setContiguousBlocks(newlist);
-      return uc;
-    } else {
-      assert hasNoContiguousBlock();
-      return sb.removeLastBlock(oldblock);
+    if (blocks == null || blocks.length == 0) {
+      return null;
+    }
+    int size_1 = blocks.length - 1;
+    if (!blocks[size_1].equals(oldblock)) {
+      return null;
     }
     }
+
+    BlockInfoUnderConstruction uc =
+        (BlockInfoUnderConstruction)blocks[size_1];
+    //copy to a new list
+    BlockInfo[] newlist = new BlockInfo[size_1];
+    System.arraycopy(blocks, 0, newlist, 0, size_1);
+    setBlocks(newlist);
+    return uc;
   }
   }
 
 
   /* End of Under-Construction Feature */
   /* End of Under-Construction Feature */
@@ -401,7 +380,7 @@ public class INodeFile extends INodeWithAdditionalFields
 
 
   /** The same as getFileReplication(null). */
   /** The same as getFileReplication(null). */
   @Override // INodeFileAttributes
   @Override // INodeFileAttributes
-  // TODO striped
+  // TODO properly handle striped files
   public final short getFileReplication() {
   public final short getFileReplication() {
     return getFileReplication(CURRENT_STATE_ID);
     return getFileReplication(CURRENT_STATE_ID);
   }
   }
@@ -466,6 +445,16 @@ public class INodeFile extends INodeWithAdditionalFields
     setStoragePolicyID(storagePolicyId);
     setStoragePolicyID(storagePolicyId);
   }
   }
 
 
+
+  /**
+   * @return true if the file is in the striping layout.
+   */
+  @VisibleForTesting
+  @Override
+  public boolean isStriped() {
+    return HeaderFormat.isStriped(header);
+  }
+
   @Override // INodeFileAttributes
   @Override // INodeFileAttributes
   public long getHeaderLong() {
   public long getHeaderLong() {
     return header;
     return header;
@@ -474,17 +463,6 @@ public class INodeFile extends INodeWithAdditionalFields
   /** @return the blocks of the file. */
   /** @return the blocks of the file. */
   @Override // BlockCollection
   @Override // BlockCollection
   public BlockInfo[] getBlocks() {
   public BlockInfo[] getBlocks() {
-    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
-    if (sb != null) {
-      assert hasNoContiguousBlock();
-      return sb.getBlocks();
-    } else {
-      return this.blocks;
-    }
-  }
-
-  /** Used by snapshot diff */
-  public BlockInfoContiguous[] getContiguousBlocks() {
     return this.blocks;
     return this.blocks;
   }
   }
 
 
@@ -507,16 +485,11 @@ public class INodeFile extends INodeWithAdditionalFields
   }
   }
 
 
   /** Used during concat to update the BlockCollection for each block */
   /** Used during concat to update the BlockCollection for each block */
-  private void updateBlockCollection() {
-    if (blocks != null && blocks.length > 0) {
-      for(BlockInfoContiguous b : blocks) {
+  void updateBlockCollection() {
+    if (blocks != null) {
+      for(BlockInfo b : blocks) {
         b.setBlockCollection(this);
         b.setBlockCollection(this);
       }
       }
-    } else {
-      FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
-      if (sb != null) {
-        sb.updateBlockCollection(this);
-      }
     }
     }
   }
   }
 
 
@@ -527,11 +500,12 @@ public class INodeFile extends INodeWithAdditionalFields
     int size = this.blocks.length;
     int size = this.blocks.length;
     int totalAddedBlocks = 0;
     int totalAddedBlocks = 0;
     for(INodeFile f : inodes) {
     for(INodeFile f : inodes) {
+      Preconditions.checkState(f.isStriped() == this.isStriped());
       totalAddedBlocks += f.blocks.length;
       totalAddedBlocks += f.blocks.length;
     }
     }
     
     
-    BlockInfoContiguous[] newlist =
-        new BlockInfoContiguous[size + totalAddedBlocks];
+    BlockInfo[] newlist =
+        new BlockInfo[size + totalAddedBlocks];
     System.arraycopy(this.blocks, 0, newlist, 0, size);
     System.arraycopy(this.blocks, 0, newlist, 0, size);
     
     
     for(INodeFile in: inodes) {
     for(INodeFile in: inodes) {
@@ -539,43 +513,35 @@ public class INodeFile extends INodeWithAdditionalFields
       size += in.blocks.length;
       size += in.blocks.length;
     }
     }
 
 
-    setContiguousBlocks(newlist);
+    setBlocks(newlist);
     updateBlockCollection();
     updateBlockCollection();
   }
   }
   
   
   /**
   /**
-   * add a contiguous block to the block list
+   * add a block to the block list
    */
    */
-  private void addContiguousBlock(BlockInfoContiguous newblock) {
+  void addBlock(BlockInfo newblock) {
+    Preconditions.checkArgument(newblock.isStriped() == this.isStriped());
     if (this.blocks == null) {
     if (this.blocks == null) {
-      this.setContiguousBlocks(new BlockInfoContiguous[]{newblock});
+      this.setBlocks(new BlockInfo[]{newblock});
     } else {
     } else {
       int size = this.blocks.length;
       int size = this.blocks.length;
-      BlockInfoContiguous[] newlist = new BlockInfoContiguous[size + 1];
+      BlockInfo[] newlist = new BlockInfo[size + 1];
       System.arraycopy(this.blocks, 0, newlist, 0, size);
       System.arraycopy(this.blocks, 0, newlist, 0, size);
       newlist[size] = newblock;
       newlist[size] = newblock;
-      this.setContiguousBlocks(newlist);
-    }
-  }
-
-  /** 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);
+      this.setBlocks(newlist);
     }
     }
   }
   }
 
 
   /** Set the blocks. */
   /** Set the blocks. */
-  public void setContiguousBlocks(BlockInfoContiguous[] blocks) {
+  private void setBlocks(BlockInfo[] blocks) {
     this.blocks = blocks;
     this.blocks = blocks;
   }
   }
 
 
+  public void clearBlocks() {
+    setBlocks(null);
+  }
+
   @Override
   @Override
   public void cleanSubtree(ReclaimContext reclaimContext,
   public void cleanSubtree(ReclaimContext reclaimContext,
       final int snapshot, int priorSnapshotId) {
       final int snapshot, int priorSnapshotId) {
@@ -623,19 +589,13 @@ public class INodeFile extends INodeWithAdditionalFields
   }
   }
 
 
   public void clearFile(ReclaimContext reclaimContext) {
   public void clearFile(ReclaimContext reclaimContext) {
-    BlockInfo[] blks = getBlocks();
-    if (blks != null && reclaimContext.collectedBlocks != null) {
-      for (BlockInfo blk : blks) {
+    if (blocks != null && reclaimContext.collectedBlocks != null) {
+      for (BlockInfo blk : blocks) {
         reclaimContext.collectedBlocks.addDeleteBlock(blk);
         reclaimContext.collectedBlocks.addDeleteBlock(blk);
         blk.setBlockCollection(null);
         blk.setBlockCollection(null);
       }
       }
     }
     }
-    setContiguousBlocks(null);
-
-    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
-    if (sb != null) {
-      sb.clear();
-    }
+    clearBlocks();
     if (getAclFeature() != null) {
     if (getAclFeature() != null) {
       AclStorage.removeAclFeature(getAclFeature());
       AclStorage.removeAclFeature(getAclFeature());
     }
     }
@@ -677,7 +637,7 @@ public class INodeFile extends INodeWithAdditionalFields
     if (isStriped()) {
     if (isStriped()) {
       return computeQuotaUsageWithStriped(bsp, counts);
       return computeQuotaUsageWithStriped(bsp, counts);
     }
     }
-    
+
     if (last < lastSnapshotId) {
     if (last < lastSnapshotId) {
       ssDeltaNoReplication = computeFileSize(true, false);
       ssDeltaNoReplication = computeFileSize(true, false);
       replication = getFileReplication();
       replication = getFileReplication();
@@ -790,33 +750,27 @@ public class INodeFile extends INodeWithAdditionalFields
    * @return file size
    * @return file size
    */
    */
   public final long computeFileSize(boolean includesLastUcBlock,
   public final long computeFileSize(boolean includesLastUcBlock,
-                                    boolean usePreferredBlockSize4LastUcBlock) {
-    BlockInfo[] blockInfos = getBlocks();
-    // In case of contiguous blocks
-    if (blockInfos == null || blockInfos.length == 0) {
+      boolean usePreferredBlockSize4LastUcBlock) {
+    if (blocks == null || blocks.length == 0) {
       return 0;
       return 0;
     }
     }
-    final int last = blockInfos.length - 1;
+    final int last = blocks.length - 1;
     //check if the last block is BlockInfoUnderConstruction
     //check if the last block is BlockInfoUnderConstruction
-    long size = blockInfos[last].getNumBytes();
-    if (blockInfos[last] instanceof BlockInfoContiguousUnderConstruction) {
-      if (!includesLastUcBlock) {
-        size = 0;
-      } else if (usePreferredBlockSize4LastUcBlock) {
-        size = getPreferredBlockSize();
-      }
-    } else if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
-      if (!includesLastUcBlock) {
-        size = 0;
-      } else if (usePreferredBlockSize4LastUcBlock) {
-        BlockInfoStripedUnderConstruction blockInfoStripedUC
-            = (BlockInfoStripedUnderConstruction) blockInfos[last];
-        size = getPreferredBlockSize() * blockInfoStripedUC.getDataBlockNum();
-      }
+    BlockInfo lastBlk = blocks[last];
+    long size = lastBlk.getNumBytes();
+    if (lastBlk instanceof BlockInfoUnderConstruction) {
+       if (!includesLastUcBlock) {
+         size = 0;
+       } else if (usePreferredBlockSize4LastUcBlock) {
+         size = isStriped()?
+             getPreferredBlockSize() *
+                 ((BlockInfoStriped)lastBlk).getDataBlockNum() :
+             getPreferredBlockSize();
+       }
     }
     }
     //sum other blocks
     //sum other blocks
     for (int i = 0; i < last; i++) {
     for (int i = 0; i < last; i++) {
-      size += blockInfos[i].getNumBytes();
+      size += blocks[i].getNumBytes();
     }
     }
     return size;
     return size;
   }
   }
@@ -828,41 +782,30 @@ public class INodeFile extends INodeWithAdditionalFields
    */
    */
   public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) {
   public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) {
     if (isStriped()) {
     if (isStriped()) {
-      return storagespaceConsumedWithStriped();
+      return storagespaceConsumedStriped();
     } else {
     } else {
-      return storagespaceConsumedWithReplication(bsp);
+      return storagespaceConsumedContiguous(bsp);
     }
     }
   }
   }
 
 
   // TODO: support EC with heterogeneous storage
   // TODO: support EC with heterogeneous storage
-  public final QuotaCounts storagespaceConsumedWithStriped() {
+  public final QuotaCounts storagespaceConsumedStriped() {
     QuotaCounts counts = new QuotaCounts.Builder().build();
     QuotaCounts counts = new QuotaCounts.Builder().build();
-    BlockInfo[] blockInfos = getBlocks();
-    if (blockInfos == null || blockInfos.length == 0) {
+    if (blocks == null || blocks.length == 0) {
       return counts;
       return counts;
     }
     }
 
 
-    long size;
-    final int last = blockInfos.length - 1;
-    if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
-      BlockInfoStripedUnderConstruction blockInfoStripedUC
-          =(BlockInfoStripedUnderConstruction)blockInfos[last];
-      size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum();
-    } else {
-      // In case of last block is complete
-      BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[last];
-      size = blockInfoStriped.spaceConsumed();
-    }
-    for (int i = 0; i < last; i++) {
-      BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[i];
-      size += blockInfoStriped.spaceConsumed();
+    for (BlockInfo b : blocks) {
+      Preconditions.checkState(b.isStriped());
+      long blockSize = b.isComplete() ?
+          ((BlockInfoStriped)b).spaceConsumed() : getPreferredBlockSize() *
+          ((BlockInfoStriped)b).getTotalBlockNum();
+      counts.addStorageSpace(blockSize);
     }
     }
-
-    counts.addStorageSpace(size);
     return  counts;
     return  counts;
   }
   }
 
 
-  public final QuotaCounts storagespaceConsumedWithReplication(
+  public final QuotaCounts storagespaceConsumedContiguous(
       BlockStoragePolicy bsp) {
       BlockStoragePolicy bsp) {
     QuotaCounts counts = new QuotaCounts.Builder().build();
     QuotaCounts counts = new QuotaCounts.Builder().build();
     final Iterable<BlockInfo> blocks;
     final Iterable<BlockInfo> blocks;
@@ -874,7 +817,7 @@ public class INodeFile extends INodeWithAdditionalFields
       Set<BlockInfo> allBlocks = new HashSet<>(Arrays.asList(getBlocks()));
       Set<BlockInfo> allBlocks = new HashSet<>(Arrays.asList(getBlocks()));
       List<FileDiff> diffs = sf.getDiffs().asList();
       List<FileDiff> diffs = sf.getDiffs().asList();
       for(FileDiff diff : diffs) {
       for(FileDiff diff : diffs) {
-        BlockInfoContiguous[] diffBlocks = diff.getBlocks();
+        BlockInfo[] diffBlocks = diff.getBlocks();
         if (diffBlocks != null) {
         if (diffBlocks != null) {
           allBlocks.addAll(Arrays.asList(diffBlocks));
           allBlocks.addAll(Arrays.asList(diffBlocks));
         }
         }
@@ -903,32 +846,20 @@ public class INodeFile extends INodeWithAdditionalFields
    * Return the penultimate allocated block for this file.
    * Return the penultimate allocated block for this file.
    */
    */
   BlockInfo getPenultimateBlock() {
   BlockInfo getPenultimateBlock() {
-    BlockInfo[] blks = getBlocks();
-    return (blks == null || blks.length <= 1) ?
-        null : blks[blks.length - 2];
+    if (blocks == null || blocks.length <= 1) {
+      return null;
+    }
+    return blocks[blocks.length - 2];
   }
   }
 
 
   @Override
   @Override
   public BlockInfo getLastBlock() {
   public BlockInfo getLastBlock() {
-    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
-    if (sb == null) {
-      return blocks == null || blocks.length == 0 ?
-          null : blocks[blocks.length - 1];
-    } else {
-      assert hasNoContiguousBlock();
-      return sb.getLastBlock();
-    }
+    return blocks == null || blocks.length == 0? null: blocks[blocks.length-1];
   }
   }
 
 
   @Override
   @Override
   public int numBlocks() {
   public int numBlocks() {
-    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
-    if (sb == null) {
-      return blocks == null ? 0 : blocks.length;
-    } else {
-      assert hasNoContiguousBlock();
-      return sb.numBlocks();
-    }
+    return blocks == null ? 0 : blocks.length;
   }
   }
 
 
   @VisibleForTesting
   @VisibleForTesting
@@ -939,8 +870,7 @@ public class INodeFile extends INodeWithAdditionalFields
     out.print(", fileSize=" + computeFileSize(snapshotId));
     out.print(", fileSize=" + computeFileSize(snapshotId));
     // only compare the first block
     // only compare the first block
     out.print(", blocks=");
     out.print(", blocks=");
-    BlockInfo[] blks = getBlocks();
-    out.print(blks == null || blks.length == 0? null: blks[0]);
+    out.print(blocks == null || blocks.length == 0? null: blocks[0]);
     out.println();
     out.println();
   }
   }
 
 
@@ -1033,24 +963,15 @@ public class INodeFile extends INodeWithAdditionalFields
   }
   }
 
 
   void truncateBlocksTo(int n) {
   void truncateBlocksTo(int n) {
-    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
-    if (sb == null) {
-      truncateContiguousBlocks(n);
-    } else {
-      sb.truncateStripedBlocks(n);
-    }
-  }
-
-  private void truncateContiguousBlocks(int n) {
-    final BlockInfoContiguous[] newBlocks;
+    final BlockInfo[] newBlocks;
     if (n == 0) {
     if (n == 0) {
-      newBlocks = BlockInfoContiguous.EMPTY_ARRAY;
+      newBlocks = BlockInfo.EMPTY_ARRAY;
     } else {
     } else {
-      newBlocks = new BlockInfoContiguous[n];
-      System.arraycopy(blocks, 0, newBlocks, 0, n);
+      newBlocks = new BlockInfo[n];
+      System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
     }
     }
     // set new blocks
     // set new blocks
-    setContiguousBlocks(newBlocks);
+    setBlocks(newBlocks);
   }
   }
 
 
   /**
   /**
@@ -1059,10 +980,11 @@ public class INodeFile extends INodeWithAdditionalFields
    * snapshots. Since we do not support truncation with striped blocks,
    * snapshots. Since we do not support truncation with striped blocks,
    * we only need to handle contiguous blocks here.
    * we only need to handle contiguous blocks here.
    */
    */
-  public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks,
+  public void collectBlocksBeyondSnapshot(BlockInfo[] snapshotBlocks,
                                           BlocksMapUpdateInfo collectedBlocks) {
                                           BlocksMapUpdateInfo collectedBlocks) {
-    BlockInfoContiguous[] oldBlocks = this.blocks;
-    if (snapshotBlocks == null || oldBlocks == null)
+    Preconditions.checkState(!isStriped());
+    BlockInfo[] oldBlocks = getBlocks();
+    if(snapshotBlocks == null || oldBlocks == null)
       return;
       return;
     // Skip blocks in common between the file and the snapshot
     // Skip blocks in common between the file and the snapshot
     int n = 0;
     int n = 0;
@@ -1070,7 +992,7 @@ public class INodeFile extends INodeWithAdditionalFields
           oldBlocks[n] == snapshotBlocks[n]) {
           oldBlocks[n] == snapshotBlocks[n]) {
       n++;
       n++;
     }
     }
-    truncateContiguousBlocks(n);
+    truncateBlocksTo(n);
     // Collect the remaining blocks of the file
     // Collect the remaining blocks of the file
     while(n < oldBlocks.length) {
     while(n < oldBlocks.length) {
       collectedBlocks.addDeleteBlock(oldBlocks[n++]);
       collectedBlocks.addDeleteBlock(oldBlocks[n++]);
@@ -1085,7 +1007,7 @@ public class INodeFile extends INodeWithAdditionalFields
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if(sf == null)
     if(sf == null)
       return;
       return;
-    BlockInfoContiguous[] snapshotBlocks =
+    BlockInfo[] snapshotBlocks =
         getDiffs().findEarlierSnapshotBlocks(snapshotId);
         getDiffs().findEarlierSnapshotBlocks(snapshotId);
     if(snapshotBlocks == null)
     if(snapshotBlocks == null)
       return;
       return;
@@ -1099,23 +1021,14 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
   /**
    * @return true if the block is contained in a snapshot or false otherwise.
    * @return true if the block is contained in a snapshot or false otherwise.
    */
    */
-  boolean isBlockInLatestSnapshot(BlockInfoContiguous block) {
+  boolean isBlockInLatestSnapshot(BlockInfo block) {
     FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
     FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
     if (sf == null || sf.getDiffs() == null) {
     if (sf == null || sf.getDiffs() == null) {
       return false;
       return false;
     }
     }
-    BlockInfoContiguous[] snapshotBlocks = getDiffs()
+    BlockInfo[] snapshotBlocks = getDiffs()
         .findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
         .findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
     return snapshotBlocks != null &&
     return snapshotBlocks != null &&
         Arrays.asList(snapshotBlocks).contains(block);
         Arrays.asList(snapshotBlocks).contains(block);
   }
   }
-
-  /**
-   * @return true if the file is in the striping layout.
-   */
-  @VisibleForTesting
-  @Override
-  public boolean isStriped() {
-    return getStripedBlocksFeature() != null;
-  }
 }
 }

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

@@ -29,6 +29,9 @@ public interface INodeFileAttributes extends INodeAttributes {
   /** @return the file replication. */
   /** @return the file replication. */
   public short getFileReplication();
   public short getFileReplication();
 
 
+  /** @return whether the file is striped (instead of contiguous) */
+  public boolean isStriped();
+
   /** @return preferred block size in bytes */
   /** @return preferred block size in bytes */
   public long getPreferredBlockSize();
   public long getPreferredBlockSize();
 
 
@@ -47,10 +50,10 @@ public interface INodeFileAttributes extends INodeAttributes {
     public SnapshotCopy(byte[] name, PermissionStatus permissions,
     public SnapshotCopy(byte[] name, PermissionStatus permissions,
         AclFeature aclFeature, long modificationTime, long accessTime,
         AclFeature aclFeature, long modificationTime, long accessTime,
         short replication, long preferredBlockSize,
         short replication, long preferredBlockSize,
-        byte storagePolicyID, XAttrFeature xAttrsFeature) {
+        byte storagePolicyID, XAttrFeature xAttrsFeature, boolean isStriped) {
       super(name, permissions, aclFeature, modificationTime, accessTime, 
       super(name, permissions, aclFeature, modificationTime, accessTime, 
           xAttrsFeature);
           xAttrsFeature);
-      header = HeaderFormat.toLong(preferredBlockSize, replication,
+      header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
           storagePolicyID);
           storagePolicyID);
     }
     }
 
 
@@ -69,6 +72,11 @@ public interface INodeFileAttributes extends INodeAttributes {
       return HeaderFormat.getReplication(header);
       return HeaderFormat.getReplication(header);
     }
     }
 
 
+    @Override
+    public boolean isStriped() {
+      return HeaderFormat.isStriped(header);
+    }
+
     @Override
     @Override
     public long getPreferredBlockSize() {
     public long getPreferredBlockSize() {
       return HeaderFormat.getPreferredBlockSize(header);
       return HeaderFormat.getPreferredBlockSize(header);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java

@@ -233,7 +233,8 @@ public class FSImageFormatPBSnapshot {
               .toByteArray(), permission, acl, fileInPb.getModificationTime(),
               .toByteArray(), permission, acl, fileInPb.getModificationTime(),
               fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
               fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
               fileInPb.getPreferredBlockSize(),
               fileInPb.getPreferredBlockSize(),
-              (byte)fileInPb.getStoragePolicyID(), xAttrs);
+              (byte)fileInPb.getStoragePolicyID(), xAttrs,
+              fileInPb.getIsStriped());
         }
         }
 
 
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiff.java

@@ -21,6 +21,7 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Arrays;
 
 
+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.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -38,7 +39,7 @@ public class FileDiff extends
   /** The file size at snapshot creation time. */
   /** The file size at snapshot creation time. */
   private final long fileSize;
   private final long fileSize;
   /** A copy of the INodeFile block list. Used in truncate. */
   /** A copy of the INodeFile block list. Used in truncate. */
-  private BlockInfoContiguous[] blocks;
+  private BlockInfo[] blocks;
 
 
   FileDiff(int snapshotId, INodeFile file) {
   FileDiff(int snapshotId, INodeFile file) {
     super(snapshotId, null, null);
     super(snapshotId, null, null);
@@ -64,7 +65,7 @@ public class FileDiff extends
    * up to the current {@link #fileSize}.
    * up to the current {@link #fileSize}.
    * Should be done only once.
    * Should be done only once.
    */
    */
-  public void setBlocks(BlockInfoContiguous[] blocks) {
+  public void setBlocks(BlockInfo[] blocks) {
     if(this.blocks != null)
     if(this.blocks != null)
       return;
       return;
     int numBlocks = 0;
     int numBlocks = 0;
@@ -73,7 +74,7 @@ public class FileDiff extends
     this.blocks = Arrays.copyOf(blocks, numBlocks);
     this.blocks = Arrays.copyOf(blocks, numBlocks);
   }
   }
 
 
-  public BlockInfoContiguous[] getBlocks() {
+  public BlockInfo[] getBlocks() {
     return blocks;
     return blocks;
   }
   }
 
 
@@ -118,7 +119,7 @@ public class FileDiff extends
     if (blocks == null || collectedBlocks == null) {
     if (blocks == null || collectedBlocks == null) {
       return;
       return;
     }
     }
-    for (BlockInfoContiguous blk : blocks) {
+    for (BlockInfo blk : blocks) {
       collectedBlocks.addDeleteBlock(blk);
       collectedBlocks.addDeleteBlock(blk);
     }
     }
     blocks = null;
     blocks = null;

+ 9 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java

@@ -56,20 +56,20 @@ public class FileDiffList extends
     final FileDiff diff =
     final FileDiff diff =
         super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy);
         super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy);
     if (withBlocks) {  // Store blocks if this is the first update
     if (withBlocks) {  // Store blocks if this is the first update
-      BlockInfoContiguous[] blks = iNodeFile.getContiguousBlocks();
+      BlockInfo[] blks = iNodeFile.getBlocks();
       assert blks != null;
       assert blks != null;
       diff.setBlocks(blks);
       diff.setBlocks(blks);
     }
     }
   }
   }
 
 
-  public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) {
+  public BlockInfo[] findEarlierSnapshotBlocks(int snapshotId) {
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
       return null;
     }
     }
     List<FileDiff> diffs = this.asList();
     List<FileDiff> diffs = this.asList();
     int i = Collections.binarySearch(diffs, snapshotId);
     int i = Collections.binarySearch(diffs, snapshotId);
-    BlockInfoContiguous[] blocks = null;
+    BlockInfo[] blocks = null;
     for(i = i >= 0 ? i : -i-2; i >= 0; i--) {
     for(i = i >= 0 ? i : -i-2; i >= 0; i--) {
       blocks = diffs.get(i).getBlocks();
       blocks = diffs.get(i).getBlocks();
       if(blocks != null) {
       if(blocks != null) {
@@ -79,14 +79,14 @@ public class FileDiffList extends
     return blocks;
     return blocks;
   }
   }
 
 
-  public BlockInfoContiguous[] findLaterSnapshotBlocks(int snapshotId) {
+  public BlockInfo[] findLaterSnapshotBlocks(int snapshotId) {
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
       return null;
     }
     }
     List<FileDiff> diffs = this.asList();
     List<FileDiff> diffs = this.asList();
     int i = Collections.binarySearch(diffs, snapshotId);
     int i = Collections.binarySearch(diffs, snapshotId);
-    BlockInfoContiguous[] blocks = null;
+    BlockInfo[] blocks = null;
     for (i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
     for (i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
       blocks = diffs.get(i).getBlocks();
       blocks = diffs.get(i).getBlocks();
       if (blocks != null) {
       if (blocks != null) {
@@ -103,7 +103,7 @@ public class FileDiffList extends
    */
    */
   void combineAndCollectSnapshotBlocks(
   void combineAndCollectSnapshotBlocks(
       INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) {
       INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) {
-    BlockInfoContiguous[] removedBlocks = removed.getBlocks();
+    BlockInfo[] removedBlocks = removed.getBlocks();
     if (removedBlocks == null) {
     if (removedBlocks == null) {
       FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
       FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
       assert sf != null : "FileWithSnapshotFeature is null";
       assert sf != null : "FileWithSnapshotFeature is null";
@@ -117,11 +117,11 @@ public class FileDiffList extends
     if (earlierDiff != null) {
     if (earlierDiff != null) {
       earlierDiff.setBlocks(removedBlocks);
       earlierDiff.setBlocks(removedBlocks);
     }
     }
-    BlockInfoContiguous[] earlierBlocks =
+    BlockInfo[] earlierBlocks =
         (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
         (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
     // Find later snapshot (or file itself) with blocks
     // Find later snapshot (or file itself) with blocks
-    BlockInfoContiguous[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
-    laterBlocks = (laterBlocks == null) ? file.getContiguousBlocks() : laterBlocks;
+    BlockInfo[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
+    laterBlocks = (laterBlocks == null) ? file.getBlocks() : laterBlocks;
     // Skip blocks, which belong to either the earlier or the later lists
     // Skip blocks, which belong to either the earlier or the later lists
     int i = 0;
     int i = 0;
     for(; i < removedBlocks.length; i++) {
     for(; i < removedBlocks.length; i++) {

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

@@ -22,7 +22,7 @@ import java.util.List;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.AclFeature;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.INode;
 import org.apache.hadoop.hdfs.server.namenode.AclStorage;
 import org.apache.hadoop.hdfs.server.namenode.AclStorage;
@@ -208,7 +208,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
 
 
     // Collect blocks that should be deleted
     // Collect blocks that should be deleted
     FileDiff last = diffs.getLast();
     FileDiff last = diffs.getLast();
-    BlockInfoContiguous[] snapshotBlocks = last == null ? null : last.getBlocks();
+    BlockInfo[] snapshotBlocks = last == null ? null : last.getBlocks();
     if(snapshotBlocks == null)
     if(snapshotBlocks == null)
       file.collectBlocksBeyondMax(max, reclaimContext.collectedBlocks());
       file.collectBlocksBeyondMax(max, reclaimContext.collectedBlocks());
     else
     else

+ 2 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto

@@ -92,11 +92,6 @@ message INodeSection {
     optional string clientMachine = 2;
     optional string clientMachine = 2;
   }
   }
 
 
-  message StripedBlocksFeature {
-    // store striped blocks related information
-    optional uint32 cellSize = 1;
-  }
-
   message AclFeatureProto {
   message AclFeatureProto {
     /**
     /**
      * An ACL entry is represented by a 32-bit integer in Big Endian
      * An ACL entry is represented by a 32-bit integer in Big Endian
@@ -145,7 +140,8 @@ message INodeSection {
     optional AclFeatureProto acl = 8;
     optional AclFeatureProto acl = 8;
     optional XAttrFeatureProto xAttrs = 9;
     optional XAttrFeatureProto xAttrs = 9;
     optional uint32 storagePolicyID = 10;
     optional uint32 storagePolicyID = 10;
-    optional StripedBlocksFeature stripedBlocks = 11;
+    optional bool isStriped = 11;
+    optional uint64 stripingCellSize = 12;
   }
   }
 
 
   message QuotaByStorageTypeEntryProto {
   message QuotaByStorageTypeEntryProto {

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java

@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+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.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -66,7 +67,7 @@ public class CreateEditsLog {
     INodeDirectory dirInode = new INodeDirectory(inodeId.nextValue(), null, p,
     INodeDirectory dirInode = new INodeDirectory(inodeId.nextValue(), null, p,
       0L);
       0L);
     editLog.logMkDir(BASE_PATH, dirInode);
     editLog.logMkDir(BASE_PATH, dirInode);
-    BlockInfoContiguous[] blocks = new BlockInfoContiguous[blocksPerFile];
+    BlockInfo[] blocks = new BlockInfo[blocksPerFile];
     for (int iB = 0; iB < blocksPerFile; ++iB) {
     for (int iB = 0; iB < blocksPerFile; ++iB) {
       blocks[iB] = 
       blocks[iB] = 
        new BlockInfoContiguous(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
        new BlockInfoContiguous(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
@@ -97,7 +98,7 @@ public class CreateEditsLog {
         editLog.logMkDir(currentDir, dirInode);
         editLog.logMkDir(currentDir, dirInode);
       }
       }
       INodeFile fileUc = new INodeFile(inodeId.nextValue(), null,
       INodeFile fileUc = new INodeFile(inodeId.nextValue(), null,
-          p, 0L, 0L, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize);
+          p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
       fileUc.toUnderConstruction("", "");
       fileUc.toUnderConstruction("", "");
       editLog.logOpenFile(filePath, fileUc, false, false);
       editLog.logOpenFile(filePath, fileUc, false, false);
       editLog.logCloseFile(filePath, inode);
       editLog.logCloseFile(filePath, inode);

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

@@ -333,7 +333,7 @@ public class TestAddStripedBlocks {
     INodeFile fileNode = ns.getFSDirectory().getINode(filePath.toString()).
     INodeFile fileNode = ns.getFSDirectory().getINode(filePath.toString()).
         asFile();
         asFile();
     Assert.assertTrue(fileNode.isStriped());
     Assert.assertTrue(fileNode.isStriped());
-    BlockInfoStriped stored = fileNode.getStripedBlocksFeature().getBlocks()[0];
+    BlockInfo stored = fileNode.getBlocks()[0];
     BlockManagerTestUtil.updateState(ns.getBlockManager());
     BlockManagerTestUtil.updateState(ns.getBlockManager());
     Assert.assertEquals(0, ns.getCorruptReplicaBlocks());
     Assert.assertEquals(0, ns.getCorruptReplicaBlocks());
 
 

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

@@ -69,7 +69,7 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -205,7 +205,7 @@ public class TestEditLog {
 
 
       for (int i = 0; i < numTransactions; i++) {
       for (int i = 0; i < numTransactions; i++) {
         INodeFile inode = new INodeFile(namesystem.dir.allocateNewInodeId(), null,
         INodeFile inode = new INodeFile(namesystem.dir.allocateNewInodeId(), null,
-            p, 0L, 0L, BlockInfoContiguous.EMPTY_ARRAY, replication, blockSize);
+            p, 0L, 0L, BlockInfo.EMPTY_ARRAY, replication, blockSize);
         inode.toUnderConstruction("", "");
         inode.toUnderConstruction("", "");
 
 
         editLog.logOpenFile("/filename" + (startIndex + i), inode, false, false);
         editLog.logOpenFile("/filename" + (startIndex + i), inode, false, false);

+ 12 - 10
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingZone;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -468,7 +469,7 @@ public class TestFSEditLogLoader {
           new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize);
           new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize);
       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.addBlock(stripedBlk);
       fns.getEditLog().logAddBlock(testFilePath, file);
       fns.getEditLog().logAddBlock(testFilePath, file);
       file.toCompleteFile(System.currentTimeMillis());
       file.toCompleteFile(System.currentTimeMillis());
 
 
@@ -483,14 +484,14 @@ public class TestFSEditLogLoader {
 
 
       assertTrue(inodeLoaded.isStriped());
       assertTrue(inodeLoaded.isStriped());
 
 
-      BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
+      BlockInfo[] blks = inodeLoaded.getBlocks();
       assertEquals(1, blks.length);
       assertEquals(1, blks.length);
       assertEquals(blkId, blks[0].getBlockId());
       assertEquals(blkId, blks[0].getBlockId());
       assertEquals(blkNumBytes, blks[0].getNumBytes());
       assertEquals(blkNumBytes, blks[0].getNumBytes());
       assertEquals(timestamp, blks[0].getGenerationStamp());
       assertEquals(timestamp, blks[0].getGenerationStamp());
-      assertEquals(blockNum, blks[0].getDataBlockNum());
-      assertEquals(parityNum, blks[0].getParityBlockNum());
-      assertEquals(cellSize, blks[0].getCellSize());
+      assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum());
+      assertEquals(parityNum, ((BlockInfoStriped)blks[0]).getParityBlockNum());
+      assertEquals(cellSize, ((BlockInfoStriped)blks[0]).getCellSize());
 
 
       cluster.shutdown();
       cluster.shutdown();
       cluster = null;
       cluster = null;
@@ -536,7 +537,7 @@ public class TestFSEditLogLoader {
           new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize);
           new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize);
       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.addBlock(stripedBlk);
       fns.getEditLog().logAddBlock(testFilePath, file);
       fns.getEditLog().logAddBlock(testFilePath, file);
       file.toCompleteFile(System.currentTimeMillis());
       file.toCompleteFile(System.currentTimeMillis());
       fns.enterSafeMode(false);
       fns.enterSafeMode(false);
@@ -564,14 +565,15 @@ public class TestFSEditLogLoader {
 
 
       assertTrue(inodeLoaded.isStriped());
       assertTrue(inodeLoaded.isStriped());
 
 
-      BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
+      BlockInfo[] blks = inodeLoaded.getBlocks();
       assertEquals(1, blks.length);
       assertEquals(1, blks.length);
+      assertTrue(blks[0].isStriped());
       assertEquals(blkId, blks[0].getBlockId());
       assertEquals(blkId, blks[0].getBlockId());
       assertEquals(newBlkNumBytes, blks[0].getNumBytes());
       assertEquals(newBlkNumBytes, blks[0].getNumBytes());
       assertEquals(newTimestamp, blks[0].getGenerationStamp());
       assertEquals(newTimestamp, blks[0].getGenerationStamp());
-      assertEquals(blockNum, blks[0].getDataBlockNum());
-      assertEquals(parityNum, blks[0].getParityBlockNum());
-      assertEquals(cellSize, blks[0].getCellSize());
+      assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum());
+      assertEquals(parityNum, ((BlockInfoStriped)blks[0]).getParityBlockNum());
+      assertEquals(cellSize, ((BlockInfoStriped)blks[0]).getCellSize());
 
 
       cluster.shutdown();
       cluster.shutdown();
       cluster = null;
       cluster = null;

+ 7 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java

@@ -152,9 +152,8 @@ public class TestFSImage {
     short replication = 3;
     short replication = 3;
     long preferredBlockSize = 128*1024*1024;
     long preferredBlockSize = 128*1024*1024;
     INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
     INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
-        blks, replication, preferredBlockSize);
+        blks, replication, preferredBlockSize, (byte) 0, true);
     ByteArrayOutputStream bs = new ByteArrayOutputStream();
     ByteArrayOutputStream bs = new ByteArrayOutputStream();
-    file.addStripedBlocksFeature();
 
 
     //construct StripedBlocks for the INode
     //construct StripedBlocks for the INode
     BlockInfoStriped[] stripedBlks = new BlockInfoStriped[3];
     BlockInfoStriped[] stripedBlks = new BlockInfoStriped[3];
@@ -164,7 +163,7 @@ public class TestFSImage {
       stripedBlks[i] = new BlockInfoStriped(
       stripedBlks[i] = new BlockInfoStriped(
               new Block(stripedBlkId + i, preferredBlockSize, timestamp),
               new Block(stripedBlkId + i, preferredBlockSize, timestamp),
               testSchema, cellSize);
               testSchema, cellSize);
-      file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
+      file.addBlock(stripedBlks[i]);
     }
     }
 
 
     final String client = "testClient";
     final String client = "testClient";
@@ -206,7 +205,7 @@ public class TestFSImage {
     assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime());
     assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime());
     // TODO for striped blocks, we currently save and load them as contiguous
     // TODO for striped blocks, we currently save and load them as contiguous
     // blocks to/from legacy fsimage
     // blocks to/from legacy fsimage
-    assertEquals(3, fileByLoaded.getContiguousBlocks().length);
+    assertEquals(3, fileByLoaded.getBlocks().length);
     assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize());
     assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize());
 
 
     if (isUC) {
     if (isUC) {
@@ -405,13 +404,12 @@ public class TestFSImage {
       // check the information of striped blocks
       // check the information of striped blocks
       FSNamesystem fsn = cluster.getNamesystem();
       FSNamesystem fsn = cluster.getNamesystem();
       INodeFile inode = fsn.dir.getINode(file.toString()).asFile();
       INodeFile inode = fsn.dir.getINode(file.toString()).asFile();
-      FileWithStripedBlocksFeature sb = inode.getStripedBlocksFeature();
-      assertNotNull(sb);
-      BlockInfoStriped[] blks = sb.getBlocks();
+      assertTrue(inode.isStriped());
+      BlockInfo[] blks = inode.getBlocks();
       assertEquals(1, blks.length);
       assertEquals(1, blks.length);
       assertTrue(blks[0].isStriped());
       assertTrue(blks[0].isStriped());
-      assertEquals(HdfsConstants.NUM_DATA_BLOCKS, blks[0].getDataBlockNum());
-      assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, blks[0].getParityBlockNum());
+      assertEquals(HdfsConstants.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum());
+      assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum());
     } finally {
     } finally {
       cluster.shutdown();
       cluster.shutdown();
     }
     }

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

@@ -92,7 +92,7 @@ public class TestINodeFile {
 
 
   private static INodeFile createINodeFile(byte storagePolicyID) {
   private static INodeFile createINodeFile(byte storagePolicyID) {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-        null, (short)3, 1024L, storagePolicyID);
+        null, (short)3, 1024L, storagePolicyID, false);
   }
   }
 
 
   @Test
   @Test

+ 3 - 11
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java

@@ -57,14 +57,13 @@ public class TestStripedINodeFile {
 
 
   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, true);
   }
   }
 
 
   @Test
   @Test
   public void testBlockStripedFeature()
   public void testBlockStripedFeature()
       throws IOException, InterruptedException{
       throws IOException, InterruptedException{
     INodeFile inf = createStripedINodeFile();
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     assertTrue(inf.isStriped());
     assertTrue(inf.isStriped());
   }
   }
 
 
@@ -80,7 +79,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedLength()
   public void testBlockStripedLength()
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
     BlockInfoStriped blockInfoStriped
         = new BlockInfoStriped(blk, testSchema, cellSize);
         = new BlockInfoStriped(blk, testSchema, cellSize);
@@ -92,7 +90,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedConsumedSpace()
   public void testBlockStripedConsumedSpace()
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
     BlockInfoStriped blockInfoStriped
         = new BlockInfoStriped(blk, testSchema, cellSize);
         = new BlockInfoStriped(blk, testSchema, cellSize);
@@ -110,7 +107,7 @@ public class TestStripedINodeFile {
     //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
     //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
     //  b. <Num Bytes> % <Num Bytes per Stripes> = 1
     //  b. <Num Bytes> % <Num Bytes per Stripes> = 1
     //  c. <Last Stripe Length> * <Parity Block Num> = 1 * 3
     //  c. <Last Stripe Length> * <Parity Block Num> = 1 * 3
-    assertEquals(4, inf.storagespaceConsumedWithStriped().getStorageSpace());
+    assertEquals(4, inf.storagespaceConsumedStriped().getStorageSpace());
     assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
     assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
   }
   }
 
 
@@ -118,7 +115,6 @@ public class TestStripedINodeFile {
   public void testMultipleBlockStripedConsumedSpace()
   public void testMultipleBlockStripedConsumedSpace()
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk1 = new Block(1);
     Block blk1 = new Block(1);
     BlockInfoStriped blockInfoStriped1
     BlockInfoStriped blockInfoStriped1
         = new BlockInfoStriped(blk1, testSchema, cellSize);
         = new BlockInfoStriped(blk1, testSchema, cellSize);
@@ -130,7 +126,7 @@ public class TestStripedINodeFile {
     inf.addBlock(blockInfoStriped1);
     inf.addBlock(blockInfoStriped1);
     inf.addBlock(blockInfoStriped2);
     inf.addBlock(blockInfoStriped2);
     // This is the double size of one block in above case.
     // This is the double size of one block in above case.
-    assertEquals(4 * 2, inf.storagespaceConsumedWithStriped().getStorageSpace());
+    assertEquals(4 * 2, inf.storagespaceConsumedStriped().getStorageSpace());
     assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
     assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
   }
   }
 
 
@@ -138,7 +134,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedFileSize()
   public void testBlockStripedFileSize()
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
     BlockInfoStriped blockInfoStriped
         = new BlockInfoStriped(blk, testSchema, cellSize);
         = new BlockInfoStriped(blk, testSchema, cellSize);
@@ -154,7 +149,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedUCFileSize()
   public void testBlockStripedUCFileSize()
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStripedUnderConstruction bInfoStripedUC
     BlockInfoStripedUnderConstruction bInfoStripedUC
         = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize);
         = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize);
@@ -168,7 +162,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedComputeQuotaUsage()
   public void testBlockStripedComputeQuotaUsage()
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
     BlockInfoStriped blockInfoStriped
         = new BlockInfoStriped(blk, testSchema, cellSize);
         = new BlockInfoStriped(blk, testSchema, cellSize);
@@ -190,7 +183,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedUCComputeQuotaUsage()
   public void testBlockStripedUCComputeQuotaUsage()
       throws IOException, InterruptedException {
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     Block blk = new Block(1);
     BlockInfoStripedUnderConstruction bInfoStripedUC
     BlockInfoStripedUnderConstruction bInfoStripedUC
         = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize);
         = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize);

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

@@ -78,7 +78,7 @@ public class TestTruncateQuotaUpdate {
   @Test
   @Test
   public void testTruncateWithSnapshotNoDivergence() {
   public void testTruncateWithSnapshotNoDivergence() {
     INodeFile file = createMockFile(BLOCKSIZE * 2 + BLOCKSIZE / 2, REPLICATION);
     INodeFile file = createMockFile(BLOCKSIZE * 2 + BLOCKSIZE / 2, REPLICATION);
-    addSnapshotFeature(file, file.getContiguousBlocks());
+    addSnapshotFeature(file, file.getBlocks());
 
 
     // case 4: truncate to 1.5 blocks
     // case 4: truncate to 1.5 blocks
     // all the blocks are in snapshot. truncate need to allocate a new block
     // all the blocks are in snapshot. truncate need to allocate a new block
@@ -152,7 +152,7 @@ public class TestTruncateQuotaUpdate {
     return new BlockInfoContiguous(b, replication);
     return new BlockInfoContiguous(b, replication);
   }
   }
 
 
-  private static void addSnapshotFeature(INodeFile file, BlockInfoContiguous[] blocks) {
+  private static void addSnapshotFeature(INodeFile file, BlockInfo[] blocks) {
     FileDiff diff = mock(FileDiff.class);
     FileDiff diff = mock(FileDiff.class);
     when(diff.getBlocks()).thenReturn(blocks);
     when(diff.getBlocks()).thenReturn(blocks);
     FileDiffList diffList = new FileDiffList();
     FileDiffList diffList = new FileDiffList();