Pārlūkot izejas kodu

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

Zhe Zhang 10 gadi atpakaļ
vecāks
revīzija
7e091de136
25 mainītis faili ar 273 papildinājumiem un 494 dzēšanām
  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.
     (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
     implements LightWeightGSet.LinkedElement {
+  public static final BlockInfo[] EMPTY_ARRAY = {};
   private BlockCollection bc;
 
   /** 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
 public class BlockInfoContiguous extends BlockInfo {
-  public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
 
   public BlockInfoContiguous(short 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. */
   public void setReplication(final short oldRepl, final short newRepl,
-      final String src, final BlockInfoContiguous... blocks) {
+      final String src, final BlockInfo... blocks) {
     if (newRepl == oldRepl) {
       return;
     }
 
     // update needReplication priority queues
-    for(BlockInfoContiguous b : blocks) {
+    for(BlockInfo b : blocks) {
       updateNeededReplications(b, 0, newRepl-oldRepl);
     }
       
@@ -3051,7 +3051,7 @@ public class BlockManager {
       // old replication > the new one; need to remove copies
       LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl
           + " for " + src);
-      for(BlockInfoContiguous b : blocks) {
+      for(BlockInfo b : blocks) {
         processOverReplicatedBlock(b, newRepl, null, null);
       }
     } 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.permission.FsAction;
 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.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 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.BlockStoragePolicySuite;
 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 BlockInfoContiguous[] blocks = unprotectedSetReplication(fsd, src,
+      final BlockInfo[] blocks = unprotectedSetReplication(fsd, src,
           replication, blockRepls);
       isFile = blocks != null;
       if (isFile) {
@@ -377,7 +375,7 @@ public class FSDirAttrOp {
     }
   }
 
-  static BlockInfoContiguous[] unprotectedSetReplication(
+  static BlockInfo[] unprotectedSetReplication(
       FSDirectory fsd, String src, short replication, short[] blockRepls)
       throws QuotaExceededException, UnresolvedLinkException,
       SnapshotAccessControlException, UnsupportedActionException {
@@ -417,7 +415,7 @@ public class FSDirAttrOp {
       blockRepls[0] = oldBR;
       blockRepls[1] = newBR;
     }
-    return file.getContiguousBlocks();
+    return file.getBlocks();
   }
 
   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;
     for (INodeFile nodeToRemove : srcList) {
       if(nodeToRemove != null) {
-        nodeToRemove.setContiguousBlocks(null);
+        nodeToRemove.clearBlocks();
         nodeToRemove.getParent().removeChild(nodeToRemove);
         fsd.getINodeMap().remove(nodeToRemove);
         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.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
@@ -481,25 +480,24 @@ class FSDirWriteFileOp {
       long preferredBlockSize, boolean underConstruction, String clientName,
       String clientMachine, byte storagePolicyId) {
     final INodeFile newNode;
+    Preconditions.checkNotNull(existing);
     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 {
+      // 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);
       if (iip != null) {
-        // check if the file is in an EC zone
-        if (FSDirErasureCodingOp.isInErasureCodingZone(fsd.getFSNamesystem(),
-            iip)) {
-          newNode.addStripedBlocksFeature();
-        }
         if (aclEntries != null) {
           AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID);
         }
@@ -580,21 +578,18 @@ class FSDirWriteFileOp {
       String clientName, String clientMachine)
       throws IOException {
 
+    Preconditions.checkNotNull(existing);
     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;
     fsd.writeLock();
     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);
-      if (newiip != null
-          && FSDirErasureCodingOp.isInErasureCodingZone(fsd.getFSNamesystem(),
-              newiip)) {
-        newNode.addStripedBlocksFeature();
-      }
     } finally {
       fsd.writeUnlock();
     }
@@ -788,16 +783,18 @@ class FSDirWriteFileOp {
 
   private static INodeFile newINodeFile(
       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,
-        BlockInfoContiguous.EMPTY_ARRAY, replication, preferredBlockSize,
-        storagePolicyId);
+        BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
+        storagePolicyId, isStriped);
   }
 
   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,
-        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.server.blockmanagement.BlockInfo;
 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.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 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.MD5Hash;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.util.StringUtils;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -903,7 +900,7 @@ public class FSImageFormat {
       final long preferredBlockSize = in.readLong();
 
       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)

+ 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.INodeSection;
 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.XAttrFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.QuotaByStorageTypeEntryProto;
@@ -326,13 +325,22 @@ public final class FSImageFormatPBINode {
       INodeSection.INodeFile f = n.getFile();
       List<BlockProto> bp = f.getBlocksList();
       short replication = (short) f.getReplication();
+      boolean isStriped = f.getIsStriped();
       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(),
           n.getName().toByteArray(), permissions, f.getModificationTime(),
           f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
-          (byte)f.getStoragePolicyID());
+          (byte)f.getStoragePolicyID(), isStriped);
 
       if (f.hasAcl()) {
         int[] entries = AclEntryStatusFormat.toInt(loadAclEntries(
             f.getAcl(), state.getStringTable()));
         file.addAclFeature(new AclFeature(entries));
       }
-      
+
       if (f.hasXAttrs()) {
         file.addXAttrFeature(new XAttrFeature(
             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
       if (f.hasFileUC()) {
         INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
         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
           final BlockInfo ucBlk;
-          if (stripeFeature != null) {
+          if (isStriped) {
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
             ucBlk = new BlockInfoStripedUnderConstruction(striped,
-                striped.getSchema(), striped.getCellSize());
+                schema, (int)f.getStripingCellSize());
           } else {
             ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
                 replication);
@@ -500,7 +493,8 @@ public final class FSImageFormatPBINode {
           .setPermission(buildPermissionStatus(file, state.getStringMap()))
           .setPreferredBlockSize(file.getPreferredBlockSize())
           .setReplication(file.getFileReplication())
-          .setStoragePolicyID(file.getLocalStoragePolicyID());
+          .setStoragePolicyID(file.getLocalStoragePolicyID())
+          .setIsStriped(file.isStriped());
 
       AclFeature f = file.getAclFeature();
       if (f != null) {
@@ -654,28 +648,22 @@ public final class FSImageFormatPBINode {
     private void save(OutputStream out, INodeFile n) throws IOException {
       INodeSection.INodeFile.Builder b = buildINodeFile(n,
           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));
         }
       }
 
-      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();

+ 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.BlockInfo;
 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.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
@@ -84,12 +82,14 @@ public class INodeFile extends INodeWithAdditionalFields
 
   /** 
    * 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 {
     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);
 
     private final LongBitFormat BITS;
@@ -110,14 +110,27 @@ public class INodeFile extends INodeWithAdditionalFields
       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,
-        byte storagePolicyID) {
+        boolean isStriped, byte storagePolicyID) {
       long h = 0;
       if (preferredBlockSize == 0) {
         preferredBlockSize = PREFERRED_BLOCK_SIZE.BITS.getMin();
       }
       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);
       return h;
     }
@@ -126,21 +139,27 @@ public class INodeFile extends INodeWithAdditionalFields
 
   private long header = 0L;
 
-  private BlockInfoContiguous[] blocks;
+  private BlockInfo[] blocks;
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
-            long atime, BlockInfoContiguous[] blklist, short replication,
+            long atime, BlockInfo[] blklist, short replication,
             long preferredBlockSize) {
     this(id, name, permissions, mtime, atime, blklist, replication,
-         preferredBlockSize, (byte) 0);
+        preferredBlockSize, (byte) 0, false);
   }
 
   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);
-    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) {
@@ -177,27 +196,6 @@ public class INodeFile extends INodeWithAdditionalFields
         && 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 */
 
   /**
@@ -232,7 +230,7 @@ public class INodeFile extends INodeWithAdditionalFields
         "file is no longer under construction");
     FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
     if (uc != null) {
-      assertAllBlocksComplete(getBlocks());
+      assertAllBlocksComplete();
       removeFeature(uc);
       this.setModificationTime(mtime);
     }
@@ -240,32 +238,21 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** Assert all blocks are complete. */
-  private void assertAllBlocksComplete(BlockInfo[] blks) {
-    if (blks == null) {
+  private void assertAllBlocksComplete() {
+    if (blocks == null) {
       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.",
-          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
   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
@@ -278,15 +265,13 @@ public class INodeFile extends INodeWithAdditionalFields
     }
 
     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);
     } else {
-      assert hasNoContiguousBlock();
-      assert lastBlock.isStriped();
-      ucBlock = ((BlockInfoStriped) lastBlock)
+      Preconditions.checkState(!lastBlock.isStriped());
+      ucBlock = ((BlockInfoContiguous) lastBlock)
           .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
     }
     setBlock(numBlocks() - 1, ucBlock);
@@ -299,27 +284,21 @@ public class INodeFile extends INodeWithAdditionalFields
   BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
     Preconditions.checkState(isUnderConstruction(),
         "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 */
@@ -401,7 +380,7 @@ public class INodeFile extends INodeWithAdditionalFields
 
   /** The same as getFileReplication(null). */
   @Override // INodeFileAttributes
-  // TODO striped
+  // TODO properly handle striped files
   public final short getFileReplication() {
     return getFileReplication(CURRENT_STATE_ID);
   }
@@ -466,6 +445,16 @@ public class INodeFile extends INodeWithAdditionalFields
     setStoragePolicyID(storagePolicyId);
   }
 
+
+  /**
+   * @return true if the file is in the striping layout.
+   */
+  @VisibleForTesting
+  @Override
+  public boolean isStriped() {
+    return HeaderFormat.isStriped(header);
+  }
+
   @Override // INodeFileAttributes
   public long getHeaderLong() {
     return header;
@@ -474,17 +463,6 @@ public class INodeFile extends INodeWithAdditionalFields
   /** @return the blocks of the file. */
   @Override // BlockCollection
   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;
   }
 
@@ -507,16 +485,11 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** 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);
       }
-    } 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 totalAddedBlocks = 0;
     for(INodeFile f : inodes) {
+      Preconditions.checkState(f.isStriped() == this.isStriped());
       totalAddedBlocks += f.blocks.length;
     }
     
-    BlockInfoContiguous[] newlist =
-        new BlockInfoContiguous[size + totalAddedBlocks];
+    BlockInfo[] newlist =
+        new BlockInfo[size + totalAddedBlocks];
     System.arraycopy(this.blocks, 0, newlist, 0, size);
     
     for(INodeFile in: inodes) {
@@ -539,43 +513,35 @@ public class INodeFile extends INodeWithAdditionalFields
       size += in.blocks.length;
     }
 
-    setContiguousBlocks(newlist);
+    setBlocks(newlist);
     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) {
-      this.setContiguousBlocks(new BlockInfoContiguous[]{newblock});
+      this.setBlocks(new BlockInfo[]{newblock});
     } else {
       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);
       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. */
-  public void setContiguousBlocks(BlockInfoContiguous[] blocks) {
+  private void setBlocks(BlockInfo[] blocks) {
     this.blocks = blocks;
   }
 
+  public void clearBlocks() {
+    setBlocks(null);
+  }
+
   @Override
   public void cleanSubtree(ReclaimContext reclaimContext,
       final int snapshot, int priorSnapshotId) {
@@ -623,19 +589,13 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   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);
         blk.setBlockCollection(null);
       }
     }
-    setContiguousBlocks(null);
-
-    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
-    if (sb != null) {
-      sb.clear();
-    }
+    clearBlocks();
     if (getAclFeature() != null) {
       AclStorage.removeAclFeature(getAclFeature());
     }
@@ -677,7 +637,7 @@ public class INodeFile extends INodeWithAdditionalFields
     if (isStriped()) {
       return computeQuotaUsageWithStriped(bsp, counts);
     }
-    
+
     if (last < lastSnapshotId) {
       ssDeltaNoReplication = computeFileSize(true, false);
       replication = getFileReplication();
@@ -790,33 +750,27 @@ public class INodeFile extends INodeWithAdditionalFields
    * @return file size
    */
   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;
     }
-    final int last = blockInfos.length - 1;
+    final int last = blocks.length - 1;
     //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
     for (int i = 0; i < last; i++) {
-      size += blockInfos[i].getNumBytes();
+      size += blocks[i].getNumBytes();
     }
     return size;
   }
@@ -828,41 +782,30 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) {
     if (isStriped()) {
-      return storagespaceConsumedWithStriped();
+      return storagespaceConsumedStriped();
     } else {
-      return storagespaceConsumedWithReplication(bsp);
+      return storagespaceConsumedContiguous(bsp);
     }
   }
 
   // TODO: support EC with heterogeneous storage
-  public final QuotaCounts storagespaceConsumedWithStriped() {
+  public final QuotaCounts storagespaceConsumedStriped() {
     QuotaCounts counts = new QuotaCounts.Builder().build();
-    BlockInfo[] blockInfos = getBlocks();
-    if (blockInfos == null || blockInfos.length == 0) {
+    if (blocks == null || blocks.length == 0) {
       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;
   }
 
-  public final QuotaCounts storagespaceConsumedWithReplication(
+  public final QuotaCounts storagespaceConsumedContiguous(
       BlockStoragePolicy bsp) {
     QuotaCounts counts = new QuotaCounts.Builder().build();
     final Iterable<BlockInfo> blocks;
@@ -874,7 +817,7 @@ public class INodeFile extends INodeWithAdditionalFields
       Set<BlockInfo> allBlocks = new HashSet<>(Arrays.asList(getBlocks()));
       List<FileDiff> diffs = sf.getDiffs().asList();
       for(FileDiff diff : diffs) {
-        BlockInfoContiguous[] diffBlocks = diff.getBlocks();
+        BlockInfo[] diffBlocks = diff.getBlocks();
         if (diffBlocks != null) {
           allBlocks.addAll(Arrays.asList(diffBlocks));
         }
@@ -903,32 +846,20 @@ public class INodeFile extends INodeWithAdditionalFields
    * Return the penultimate allocated block for this file.
    */
   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
   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
   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
@@ -939,8 +870,7 @@ public class INodeFile extends INodeWithAdditionalFields
     out.print(", fileSize=" + computeFileSize(snapshotId));
     // only compare the first block
     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();
   }
 
@@ -1033,24 +963,15 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   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) {
-      newBlocks = BlockInfoContiguous.EMPTY_ARRAY;
+      newBlocks = BlockInfo.EMPTY_ARRAY;
     } 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
-    setContiguousBlocks(newBlocks);
+    setBlocks(newBlocks);
   }
 
   /**
@@ -1059,10 +980,11 @@ public class INodeFile extends INodeWithAdditionalFields
    * snapshots. Since we do not support truncation with striped blocks,
    * we only need to handle contiguous blocks here.
    */
-  public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks,
+  public void collectBlocksBeyondSnapshot(BlockInfo[] snapshotBlocks,
                                           BlocksMapUpdateInfo collectedBlocks) {
-    BlockInfoContiguous[] oldBlocks = this.blocks;
-    if (snapshotBlocks == null || oldBlocks == null)
+    Preconditions.checkState(!isStriped());
+    BlockInfo[] oldBlocks = getBlocks();
+    if(snapshotBlocks == null || oldBlocks == null)
       return;
     // Skip blocks in common between the file and the snapshot
     int n = 0;
@@ -1070,7 +992,7 @@ public class INodeFile extends INodeWithAdditionalFields
           oldBlocks[n] == snapshotBlocks[n]) {
       n++;
     }
-    truncateContiguousBlocks(n);
+    truncateBlocksTo(n);
     // Collect the remaining blocks of the file
     while(n < oldBlocks.length) {
       collectedBlocks.addDeleteBlock(oldBlocks[n++]);
@@ -1085,7 +1007,7 @@ public class INodeFile extends INodeWithAdditionalFields
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if(sf == null)
       return;
-    BlockInfoContiguous[] snapshotBlocks =
+    BlockInfo[] snapshotBlocks =
         getDiffs().findEarlierSnapshotBlocks(snapshotId);
     if(snapshotBlocks == null)
       return;
@@ -1099,23 +1021,14 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * @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();
     if (sf == null || sf.getDiffs() == null) {
       return false;
     }
-    BlockInfoContiguous[] snapshotBlocks = getDiffs()
+    BlockInfo[] snapshotBlocks = getDiffs()
         .findEarlierSnapshotBlocks(getDiffs().getLastSnapshotId());
     return snapshotBlocks != null &&
         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. */
   public short getFileReplication();
 
+  /** @return whether the file is striped (instead of contiguous) */
+  public boolean isStriped();
+
   /** @return preferred block size in bytes */
   public long getPreferredBlockSize();
 
@@ -47,10 +50,10 @@ public interface INodeFileAttributes extends INodeAttributes {
     public SnapshotCopy(byte[] name, PermissionStatus permissions,
         AclFeature aclFeature, long modificationTime, long accessTime,
         short replication, long preferredBlockSize,
-        byte storagePolicyID, XAttrFeature xAttrsFeature) {
+        byte storagePolicyID, XAttrFeature xAttrsFeature, boolean isStriped) {
       super(name, permissions, aclFeature, modificationTime, accessTime, 
           xAttrsFeature);
-      header = HeaderFormat.toLong(preferredBlockSize, replication,
+      header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
           storagePolicyID);
     }
 
@@ -69,6 +72,11 @@ public interface INodeFileAttributes extends INodeAttributes {
       return HeaderFormat.getReplication(header);
     }
 
+    @Override
+    public boolean isStriped() {
+      return HeaderFormat.isStriped(header);
+    }
+
     @Override
     public long getPreferredBlockSize() {
       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(),
               fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
               fileInPb.getPreferredBlockSize(),
-              (byte)fileInPb.getStoragePolicyID(), xAttrs);
+              (byte)fileInPb.getStoragePolicyID(), xAttrs,
+              fileInPb.getIsStriped());
         }
 
         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.util.Arrays;
 
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INode;
@@ -38,7 +39,7 @@ public class FileDiff extends
   /** The file size at snapshot creation time. */
   private final long fileSize;
   /** A copy of the INodeFile block list. Used in truncate. */
-  private BlockInfoContiguous[] blocks;
+  private BlockInfo[] blocks;
 
   FileDiff(int snapshotId, INodeFile file) {
     super(snapshotId, null, null);
@@ -64,7 +65,7 @@ public class FileDiff extends
    * up to the current {@link #fileSize}.
    * Should be done only once.
    */
-  public void setBlocks(BlockInfoContiguous[] blocks) {
+  public void setBlocks(BlockInfo[] blocks) {
     if(this.blocks != null)
       return;
     int numBlocks = 0;
@@ -73,7 +74,7 @@ public class FileDiff extends
     this.blocks = Arrays.copyOf(blocks, numBlocks);
   }
 
-  public BlockInfoContiguous[] getBlocks() {
+  public BlockInfo[] getBlocks() {
     return blocks;
   }
 
@@ -118,7 +119,7 @@ public class FileDiff extends
     if (blocks == null || collectedBlocks == null) {
       return;
     }
-    for (BlockInfoContiguous blk : blocks) {
+    for (BlockInfo blk : blocks) {
       collectedBlocks.addDeleteBlock(blk);
     }
     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 =
         super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy);
     if (withBlocks) {  // Store blocks if this is the first update
-      BlockInfoContiguous[] blks = iNodeFile.getContiguousBlocks();
+      BlockInfo[] blks = iNodeFile.getBlocks();
       assert blks != null;
       diff.setBlocks(blks);
     }
   }
 
-  public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) {
+  public BlockInfo[] findEarlierSnapshotBlocks(int snapshotId) {
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
     }
     List<FileDiff> diffs = this.asList();
     int i = Collections.binarySearch(diffs, snapshotId);
-    BlockInfoContiguous[] blocks = null;
+    BlockInfo[] blocks = null;
     for(i = i >= 0 ? i : -i-2; i >= 0; i--) {
       blocks = diffs.get(i).getBlocks();
       if(blocks != null) {
@@ -79,14 +79,14 @@ public class FileDiffList extends
     return blocks;
   }
 
-  public BlockInfoContiguous[] findLaterSnapshotBlocks(int snapshotId) {
+  public BlockInfo[] findLaterSnapshotBlocks(int snapshotId) {
     assert snapshotId != Snapshot.NO_SNAPSHOT_ID : "Wrong snapshot id";
     if (snapshotId == Snapshot.CURRENT_STATE_ID) {
       return null;
     }
     List<FileDiff> diffs = this.asList();
     int i = Collections.binarySearch(diffs, snapshotId);
-    BlockInfoContiguous[] blocks = null;
+    BlockInfo[] blocks = null;
     for (i = i >= 0 ? i+1 : -i-1; i < diffs.size(); i++) {
       blocks = diffs.get(i).getBlocks();
       if (blocks != null) {
@@ -103,7 +103,7 @@ public class FileDiffList extends
    */
   void combineAndCollectSnapshotBlocks(
       INode.ReclaimContext reclaimContext, INodeFile file, FileDiff removed) {
-    BlockInfoContiguous[] removedBlocks = removed.getBlocks();
+    BlockInfo[] removedBlocks = removed.getBlocks();
     if (removedBlocks == null) {
       FileWithSnapshotFeature sf = file.getFileWithSnapshotFeature();
       assert sf != null : "FileWithSnapshotFeature is null";
@@ -117,11 +117,11 @@ public class FileDiffList extends
     if (earlierDiff != null) {
       earlierDiff.setBlocks(removedBlocks);
     }
-    BlockInfoContiguous[] earlierBlocks =
+    BlockInfo[] earlierBlocks =
         (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
     // 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
     int i = 0;
     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.fs.StorageType;
 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.INode;
 import org.apache.hadoop.hdfs.server.namenode.AclStorage;
@@ -208,7 +208,7 @@ public class FileWithSnapshotFeature implements INode.Feature {
 
     // Collect blocks that should be deleted
     FileDiff last = diffs.getLast();
-    BlockInfoContiguous[] snapshotBlocks = last == null ? null : last.getBlocks();
+    BlockInfo[] snapshotBlocks = last == null ? null : last.getBlocks();
     if(snapshotBlocks == null)
       file.collectBlocksBeyondMax(max, reclaimContext.collectedBlocks());
     else

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

@@ -92,11 +92,6 @@ message INodeSection {
     optional string clientMachine = 2;
   }
 
-  message StripedBlocksFeature {
-    // store striped blocks related information
-    optional uint32 cellSize = 1;
-  }
-
   message AclFeatureProto {
     /**
      * An ACL entry is represented by a 32-bit integer in Big Endian
@@ -145,7 +140,8 @@ message INodeSection {
     optional AclFeatureProto acl = 8;
     optional XAttrFeatureProto xAttrs = 9;
     optional uint32 storagePolicyID = 10;
-    optional StripedBlocksFeature stripedBlocks = 11;
+    optional bool isStriped = 11;
+    optional uint64 stripingCellSize = 12;
   }
 
   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.PermissionStatus;
 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.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.Storage;
@@ -66,7 +67,7 @@ public class CreateEditsLog {
     INodeDirectory dirInode = new INodeDirectory(inodeId.nextValue(), null, p,
       0L);
     editLog.logMkDir(BASE_PATH, dirInode);
-    BlockInfoContiguous[] blocks = new BlockInfoContiguous[blocksPerFile];
+    BlockInfo[] blocks = new BlockInfo[blocksPerFile];
     for (int iB = 0; iB < blocksPerFile; ++iB) {
       blocks[iB] = 
        new BlockInfoContiguous(new Block(0, blockSize, BLOCK_GENERATION_STAMP),
@@ -97,7 +98,7 @@ public class CreateEditsLog {
         editLog.logMkDir(currentDir, dirInode);
       }
       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("", "");
       editLog.logOpenFile(filePath, fileUc, false, false);
       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()).
         asFile();
     Assert.assertTrue(fileNode.isStriped());
-    BlockInfoStriped stored = fileNode.getStripedBlocksFeature().getBlocks()[0];
+    BlockInfo stored = fileNode.getBlocks()[0];
     BlockManagerTestUtil.updateState(ns.getBlockManager());
     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.MiniDFSCluster;
 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.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
@@ -205,7 +205,7 @@ public class TestEditLog {
 
       for (int i = 0; i < numTransactions; i++) {
         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("", "");
 
         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.MiniDFSCluster;
 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.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -468,7 +469,7 @@ public class TestFSEditLogLoader {
           new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize);
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       file.toUnderConstruction(clientName, clientMachine);
-      file.getStripedBlocksFeature().addBlock(stripedBlk);
+      file.addBlock(stripedBlk);
       fns.getEditLog().logAddBlock(testFilePath, file);
       file.toCompleteFile(System.currentTimeMillis());
 
@@ -483,14 +484,14 @@ public class TestFSEditLogLoader {
 
       assertTrue(inodeLoaded.isStriped());
 
-      BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
+      BlockInfo[] blks = inodeLoaded.getBlocks();
       assertEquals(1, blks.length);
       assertEquals(blkId, blks[0].getBlockId());
       assertEquals(blkNumBytes, blks[0].getNumBytes());
       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 = null;
@@ -536,7 +537,7 @@ public class TestFSEditLogLoader {
           new Block(blkId, blkNumBytes, timestamp), testSchema, cellSize);
       INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
       file.toUnderConstruction(clientName, clientMachine);
-      file.getStripedBlocksFeature().addBlock(stripedBlk);
+      file.addBlock(stripedBlk);
       fns.getEditLog().logAddBlock(testFilePath, file);
       file.toCompleteFile(System.currentTimeMillis());
       fns.enterSafeMode(false);
@@ -564,14 +565,15 @@ public class TestFSEditLogLoader {
 
       assertTrue(inodeLoaded.isStriped());
 
-      BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
+      BlockInfo[] blks = inodeLoaded.getBlocks();
       assertEquals(1, blks.length);
+      assertTrue(blks[0].isStriped());
       assertEquals(blkId, blks[0].getBlockId());
       assertEquals(newBlkNumBytes, blks[0].getNumBytes());
       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 = 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;
     long preferredBlockSize = 128*1024*1024;
     INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
-        blks, replication, preferredBlockSize);
+        blks, replication, preferredBlockSize, (byte) 0, true);
     ByteArrayOutputStream bs = new ByteArrayOutputStream();
-    file.addStripedBlocksFeature();
 
     //construct StripedBlocks for the INode
     BlockInfoStriped[] stripedBlks = new BlockInfoStriped[3];
@@ -164,7 +163,7 @@ public class TestFSImage {
       stripedBlks[i] = new BlockInfoStriped(
               new Block(stripedBlkId + i, preferredBlockSize, timestamp),
               testSchema, cellSize);
-      file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
+      file.addBlock(stripedBlks[i]);
     }
 
     final String client = "testClient";
@@ -206,7 +205,7 @@ public class TestFSImage {
     assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime());
     // TODO for striped blocks, we currently save and load them as contiguous
     // blocks to/from legacy fsimage
-    assertEquals(3, fileByLoaded.getContiguousBlocks().length);
+    assertEquals(3, fileByLoaded.getBlocks().length);
     assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize());
 
     if (isUC) {
@@ -405,13 +404,12 @@ public class TestFSImage {
       // check the information of striped blocks
       FSNamesystem fsn = cluster.getNamesystem();
       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);
       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 {
       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) {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-        null, (short)3, 1024L, storagePolicyID);
+        null, (short)3, 1024L, storagePolicyID, false);
   }
 
   @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() {
     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
   public void testBlockStripedFeature()
       throws IOException, InterruptedException{
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     assertTrue(inf.isStriped());
   }
 
@@ -80,7 +79,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedLength()
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
         = new BlockInfoStriped(blk, testSchema, cellSize);
@@ -92,7 +90,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedConsumedSpace()
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
         = new BlockInfoStriped(blk, testSchema, cellSize);
@@ -110,7 +107,7 @@ public class TestStripedINodeFile {
     //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
     //  b. <Num Bytes> % <Num Bytes per Stripes> = 1
     //  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());
   }
 
@@ -118,7 +115,6 @@ public class TestStripedINodeFile {
   public void testMultipleBlockStripedConsumedSpace()
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk1 = new Block(1);
     BlockInfoStriped blockInfoStriped1
         = new BlockInfoStriped(blk1, testSchema, cellSize);
@@ -130,7 +126,7 @@ public class TestStripedINodeFile {
     inf.addBlock(blockInfoStriped1);
     inf.addBlock(blockInfoStriped2);
     // 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());
   }
 
@@ -138,7 +134,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedFileSize()
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
         = new BlockInfoStriped(blk, testSchema, cellSize);
@@ -154,7 +149,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedUCFileSize()
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStripedUnderConstruction bInfoStripedUC
         = new BlockInfoStripedUnderConstruction(blk, testSchema, cellSize);
@@ -168,7 +162,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedComputeQuotaUsage()
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStriped blockInfoStriped
         = new BlockInfoStriped(blk, testSchema, cellSize);
@@ -190,7 +183,6 @@ public class TestStripedINodeFile {
   public void testBlockStripedUCComputeQuotaUsage()
       throws IOException, InterruptedException {
     INodeFile inf = createStripedINodeFile();
-    inf.addStripedBlocksFeature();
     Block blk = new Block(1);
     BlockInfoStripedUnderConstruction bInfoStripedUC
         = 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
   public void testTruncateWithSnapshotNoDivergence() {
     INodeFile file = createMockFile(BLOCKSIZE * 2 + BLOCKSIZE / 2, REPLICATION);
-    addSnapshotFeature(file, file.getContiguousBlocks());
+    addSnapshotFeature(file, file.getBlocks());
 
     // case 4: truncate to 1.5 blocks
     // 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);
   }
 
-  private static void addSnapshotFeature(INodeFile file, BlockInfoContiguous[] blocks) {
+  private static void addSnapshotFeature(INodeFile file, BlockInfo[] blocks) {
     FileDiff diff = mock(FileDiff.class);
     when(diff.getBlocks()).thenReturn(blocks);
     FileDiffList diffList = new FileDiffList();