Browse Source

HDFS-10759. Change fsimage bool isStriped from boolean to an enum. Contributed by Ewan Higgs.

Andrew Wang 8 years ago
parent
commit
a2a5d7b5bc
31 changed files with 418 additions and 109 deletions
  1. 57 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockType.java
  2. 4 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
  3. 20 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  4. 8 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto
  5. 61 0
      hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockType.java
  6. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
  7. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
  8. 12 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
  9. 5 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
  10. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
  11. 6 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
  12. 39 25
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
  13. 8 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java
  14. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java
  15. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java
  16. 29 24
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
  17. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  18. 6 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
  19. 12 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  20. 40 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
  21. 11 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
  22. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  23. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
  24. 14 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java
  25. 6 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
  26. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
  27. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
  28. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
  29. 14 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
  30. 21 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
  31. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java

+ 57 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/BlockType.java

@@ -0,0 +1,57 @@
+/**
+ * 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.protocol;
+
+/**
+ * Type of a block. Previously, all blocks were replicated (contiguous).
+ * Then Erasure Coded blocks (striped) were implemented.
+ *
+ * BlockTypes are currently defined by the highest bit in the block id. If
+ * this bit is set, then the block is striped.
+ *
+ * Further extensions may claim the second bit s.t. the highest two bits are
+ * set. e.g.
+ * 0b00 == contiguous
+ * 0b10 == striped
+ * 0b11 == possible further extension block type.
+ */
+public enum BlockType {
+  //! Replicated block.
+  CONTIGUOUS,
+  //! Erasure Coded Block
+  STRIPED;
+
+  // BLOCK_ID_MASK is the union of all masks.
+  static final long BLOCK_ID_MASK          = 1L << 63;
+  // BLOCK_ID_MASK_STRIPED is the mask for striped blocks.
+  static final long BLOCK_ID_MASK_STRIPED  = 1L << 63;
+
+  /**
+   * Parse a BlockId to find the BlockType
+   * Note: the old block id generation algorithm was based on a pseudo random
+   * number generator, so there may be legacy blocks that make this conversion
+   * unreliable.
+   */
+  public static BlockType fromBlockId(long blockId) {
+    long blockType = blockId & BLOCK_ID_MASK;
+    if(blockType == BLOCK_ID_MASK_STRIPED) {
+      return STRIPED;
+    }
+    return CONTIGUOUS;
+  }
+}

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java

@@ -211,4 +211,8 @@ public class LocatedBlock {
   public boolean isStriped() {
     return false;
   }
+
+  public BlockType getBlockType() {
+    return BlockType.CONTIGUOUS;
+  }
 }

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -60,6 +60,7 @@ import org.apache.hadoop.hdfs.inotify.EventBatch;
 import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
@@ -122,6 +123,7 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.EncryptionZon
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ContentSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
@@ -1775,6 +1777,24 @@ public class PBHelperClient {
     return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
   }
 
+  public static BlockTypeProto convert(BlockType blockType) {
+    switch (blockType) {
+    case CONTIGUOUS: return BlockTypeProto.CONTIGUOUS;
+    case STRIPED: return BlockTypeProto.STRIPED;
+    default:
+      throw new IllegalArgumentException("Unexpected block type: " + blockType);
+    }
+  }
+
+  public static BlockType convert(BlockTypeProto blockType) {
+    switch (blockType.getNumber()) {
+    case BlockTypeProto.CONTIGUOUS_VALUE: return BlockType.CONTIGUOUS;
+    case BlockTypeProto.STRIPED_VALUE: return BlockType.STRIPED;
+    default:
+      throw new IllegalArgumentException("Unexpected block type: " + blockType);
+    }
+  }
+
   static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
     if (di == null) return null;
     DatanodeInfo[] result = new DatanodeInfo[di.length];

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto

@@ -197,6 +197,14 @@ enum StorageTypeProto {
   RAM_DISK = 4;
 }
 
+/**
+ * Types of recognized blocks.
+ */
+enum BlockTypeProto {
+  CONTIGUOUS = 0;
+  STRIPED = 1;
+}
+
 /**
  * A list of storage types. 
  */

+ 61 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockType.java

@@ -0,0 +1,61 @@
+/**
+ * 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.protocol;
+
+import org.junit.Test;
+
+import static org.apache.hadoop.hdfs.protocol.BlockType.CONTIGUOUS;
+import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
+import static org.junit.Assert.*;
+
+/**
+ * Test the BlockType class.
+ */
+public class TestBlockType {
+  @Test
+  public void testGetBlockType() throws Exception {
+    assertEquals(BlockType.fromBlockId(0x0000000000000000L), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x1000000000000000L), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x2000000000000000L), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x4000000000000000L), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x7000000000000000L), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x00000000ffffffffL), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x10000000ffffffffL), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x20000000ffffffffL), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x40000000ffffffffL), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x70000000ffffffffL), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x70000000ffffffffL), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x0fffffffffffffffL), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x1fffffffffffffffL), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x2fffffffffffffffL), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x4fffffffffffffffL), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x7fffffffffffffffL), CONTIGUOUS);
+    assertEquals(BlockType.fromBlockId(0x8000000000000000L), STRIPED);
+    assertEquals(BlockType.fromBlockId(0x9000000000000000L), STRIPED);
+    assertEquals(BlockType.fromBlockId(0xa000000000000000L), STRIPED);
+    assertEquals(BlockType.fromBlockId(0xf000000000000000L), STRIPED);
+    assertEquals(BlockType.fromBlockId(0x80000000ffffffffL), STRIPED);
+    assertEquals(BlockType.fromBlockId(0x90000000ffffffffL), STRIPED);
+    assertEquals(BlockType.fromBlockId(0xa0000000ffffffffL), STRIPED);
+    assertEquals(BlockType.fromBlockId(0xf0000000ffffffffL), STRIPED);
+    assertEquals(BlockType.fromBlockId(0x8fffffffffffffffL), STRIPED);
+    assertEquals(BlockType.fromBlockId(0x9fffffffffffffffL), STRIPED);
+    assertEquals(BlockType.fromBlockId(0xafffffffffffffffL), STRIPED);
+    assertEquals(BlockType.fromBlockId(0xffffffffffffffffL), STRIPED);
+  }
+}

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.balancer;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.hdfs.protocol.BlockType.CONTIGUOUS;
 
 import java.io.IOException;
 import java.io.PrintStream;
@@ -220,7 +221,7 @@ public class Balancer {
       ) throws UnsupportedActionException {
     BlockPlacementPolicies placementPolicies =
         new BlockPlacementPolicies(conf, null, null, null);
-    if (!(placementPolicies.getPolicy(false) instanceof
+    if (!(placementPolicies.getPolicy(CONTIGUOUS) instanceof
         BlockPlacementPolicyDefault)) {
       throw new UnsupportedActionException(
           "Balancer without BlockPlacementPolicyDefault");

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java

@@ -68,6 +68,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
@@ -1274,7 +1275,7 @@ public class Dispatcher {
       }
       datanodeInfos.add(target.getDatanodeInfo());
     }
-    return placementPolicies.getPolicy(false).isMovable(
+    return placementPolicies.getPolicy(BlockType.CONTIGUOUS).isMovable(
         datanodeInfos, source.getDatanodeInfo(), target.getDatanodeInfo());
   }
 

+ 12 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java

@@ -20,12 +20,15 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 import java.io.IOException;
 
+import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
+
 /**
  * BlockIdManager allocates the generation stamps and the block ID. The
  * {@see FSNamesystem} is responsible for persisting the allocations in the
@@ -207,9 +210,14 @@ public class BlockIdManager {
   /**
    * Increments, logs and then returns the block ID
    */
-  long nextBlockId(boolean isStriped) {
-    return isStriped ? blockGroupIdGenerator.nextValue() :
-        blockIdGenerator.nextValue();
+  long nextBlockId(BlockType blockType) {
+    switch(blockType) {
+    case CONTIGUOUS: return blockIdGenerator.nextValue();
+    case STRIPED: return blockGroupIdGenerator.nextValue();
+    default:
+      throw new IllegalArgumentException(
+          "nextBlockId called with an unsupported BlockType");
+    }
   }
 
   boolean isGenStampInFuture(Block block) {
@@ -230,7 +238,7 @@ public class BlockIdManager {
   }
 
   public static boolean isStripedBlockID(long id) {
-    return id < 0;
+    return BlockType.fromBlockId(id) == STRIPED;
   }
 
   /**

+ 5 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java

@@ -25,6 +25,7 @@ import java.util.NoSuchElementException;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.LightWeightGSet;
@@ -175,6 +176,8 @@ public abstract class BlockInfo extends Block
 
   public abstract boolean isStriped();
 
+  public abstract BlockType getBlockType();
+
   /** @return true if there is no datanode storage associated with the block */
   abstract boolean hasNoStorage();
 
@@ -262,11 +265,11 @@ public abstract class BlockInfo extends Block
       DatanodeStorageInfo[] targets) {
     if (isComplete()) {
       uc = new BlockUnderConstructionFeature(this, s, targets,
-          this.isStriped());
+          this.getBlockType());
     } else {
       // the block is already under construction
       uc.setBlockUCState(s);
-      uc.setExpectedLocations(this, targets, this.isStriped());
+      uc.setExpectedLocations(this, targets, this.getBlockType());
     }
   }
 

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

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 
 /**
  * Subclass of {@link BlockInfo}, used for a block with replication scheme.
@@ -92,6 +93,11 @@ public class BlockInfoContiguous extends BlockInfo {
     return false;
   }
 
+  @Override
+  public BlockType getBlockType() {
+    return BlockType.CONTIGUOUS;
+  }
+
   @Override
   final boolean hasNoStorage() {
     return getStorageInfo(0) == null;

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

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@@ -208,6 +209,11 @@ public class BlockInfoStriped extends BlockInfo {
     return true;
   }
 
+  @Override
+  public BlockType getBlockType() {
+    return BlockType.STRIPED;
+  }
+
   @Override
   public int numNodes() {
     assert this.storages != null : "BlockInfo is not initialized";

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

@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.apache.hadoop.hdfs.protocol.BlockType.CONTIGUOUS;
+import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import java.io.IOException;
@@ -60,6 +62,7 @@ import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -642,7 +645,7 @@ public class BlockManager implements BlockStatsMXBean {
 
   @VisibleForTesting
   public BlockPlacementPolicy getBlockPlacementPolicy() {
-    return placementPolicies.getPolicy(false);
+    return placementPolicies.getPolicy(CONTIGUOUS);
   }
 
   /** Dump meta data to out. */
@@ -784,10 +787,13 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   public int getDefaultStorageNum(BlockInfo block) {
-    if (block.isStriped()) {
-      return ((BlockInfoStriped) block).getRealTotalBlockNum();
-    } else {
-      return defaultReplication;
+    switch (block.getBlockType()) {
+    case STRIPED: return ((BlockInfoStriped) block).getRealTotalBlockNum();
+    case CONTIGUOUS: return defaultReplication;
+    default:
+      throw new IllegalArgumentException(
+          "getDefaultStorageNum called with unknown BlockType: "
+          + block.getBlockType());
     }
   }
 
@@ -796,10 +802,13 @@ public class BlockManager implements BlockStatsMXBean {
   }
 
   public short getMinStorageNum(BlockInfo block) {
-    if (block.isStriped()) {
-      return ((BlockInfoStriped) block).getRealDataBlockNum();
-    } else {
-      return minReplication;
+    switch(block.getBlockType()) {
+    case STRIPED: return ((BlockInfoStriped) block).getRealDataBlockNum();
+    case CONTIGUOUS: return minReplication;
+    default:
+      throw new IllegalArgumentException(
+          "getMinStorageNum called with unknown BlockType: "
+          + block.getBlockType());
     }
   }
 
@@ -1723,7 +1732,7 @@ public class BlockManager implements BlockStatsMXBean {
       // It is costly to extract the filename for which chooseTargets is called,
       // so for now we pass in the block collection itself.
       final BlockPlacementPolicy placementPolicy =
-          placementPolicies.getPolicy(rw.getBlock().isStriped());
+          placementPolicies.getPolicy(rw.getBlock().getBlockType());
       rw.chooseTargets(placementPolicy, storagePolicySuite, excludedNodes);
     }
 
@@ -1928,9 +1937,9 @@ public class BlockManager implements BlockStatsMXBean {
   /** Choose target for WebHDFS redirection. */
   public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src,
       DatanodeDescriptor clientnode, Set<Node> excludes, long blocksize) {
-    return placementPolicies.getPolicy(false).chooseTarget(src, 1, clientnode,
-        Collections.<DatanodeStorageInfo>emptyList(), false, excludes,
-        blocksize, storagePolicySuite.getDefaultPolicy(), null);
+    return placementPolicies.getPolicy(CONTIGUOUS).chooseTarget(src, 1,
+        clientnode, Collections.<DatanodeStorageInfo>emptyList(), false,
+        excludes, blocksize, storagePolicySuite.getDefaultPolicy(), null);
   }
 
   /** Choose target for getting additional datanodes for an existing pipeline. */
@@ -1941,9 +1950,11 @@ public class BlockManager implements BlockStatsMXBean {
       Set<Node> excludes,
       long blocksize,
       byte storagePolicyID,
-      boolean isStriped) {
-    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
-    final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped);
+      BlockType blockType) {
+    final BlockStoragePolicy storagePolicy =
+        storagePolicySuite.getPolicy(storagePolicyID);
+    final BlockPlacementPolicy blockplacement =
+        placementPolicies.getPolicy(blockType);
     return blockplacement.chooseTarget(src, numAdditionalNodes, clientnode,
         chosen, true, excludes, blocksize, storagePolicy, null);
   }
@@ -1962,12 +1973,14 @@ public class BlockManager implements BlockStatsMXBean {
       final long blocksize,
       final List<String> favoredNodes,
       final byte storagePolicyID,
-      final boolean isStriped,
+      final BlockType blockType,
       final EnumSet<AddBlockFlag> flags) throws IOException {
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
         getDatanodeDescriptors(favoredNodes);
-    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID);
-    final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped);
+    final BlockStoragePolicy storagePolicy =
+        storagePolicySuite.getPolicy(storagePolicyID);
+    final BlockPlacementPolicy blockplacement =
+        placementPolicies.getPolicy(blockType);
     final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src,
         numOfReplicas, client, excludedNodes, blocksize, 
         favoredDatanodeDescriptors, storagePolicy, flags);
@@ -3428,7 +3441,7 @@ public class BlockManager implements BlockStatsMXBean {
       final Collection<DatanodeStorageInfo> nonExcess, BlockInfo storedBlock,
       short replication, DatanodeDescriptor addedNode,
       DatanodeDescriptor delNodeHint, List<StorageType> excessTypes) {
-    BlockPlacementPolicy replicator = placementPolicies.getPolicy(false);
+    BlockPlacementPolicy replicator = placementPolicies.getPolicy(CONTIGUOUS);
     List<DatanodeStorageInfo> replicasToDelete = replicator
         .chooseReplicasToDelete(nonExcess, nonExcess, replication, excessTypes,
             addedNode, delNodeHint);
@@ -3490,7 +3503,7 @@ public class BlockManager implements BlockStatsMXBean {
       return;
     }
 
-    BlockPlacementPolicy placementPolicy = placementPolicies.getPolicy(true);
+    BlockPlacementPolicy placementPolicy = placementPolicies.getPolicy(STRIPED);
     // for each duplicated index, delete some replicas until only one left
     for (int targetIndex = duplicated.nextSetBit(0); targetIndex >= 0;
          targetIndex = duplicated.nextSetBit(targetIndex + 1)) {
@@ -4145,9 +4158,10 @@ public class BlockManager implements BlockStatsMXBean {
       }
     }
     DatanodeInfo[] locs = liveNodes.toArray(new DatanodeInfo[liveNodes.size()]);
+    BlockType blockType = storedBlock.getBlockType();
     BlockPlacementPolicy placementPolicy = placementPolicies
-        .getPolicy(storedBlock.isStriped());
-    int numReplicas = storedBlock.isStriped() ? ((BlockInfoStriped) storedBlock)
+        .getPolicy(blockType);
+    int numReplicas = blockType == STRIPED ? ((BlockInfoStriped) storedBlock)
         .getRealDataBlockNum() : storedBlock.getReplication();
     return placementPolicy.verifyBlockPlacement(locs, numReplicas)
         .isPlacementPolicySatisfied();
@@ -4685,8 +4699,8 @@ public class BlockManager implements BlockStatsMXBean {
     return blockIdManager.isLegacyBlock(block);
   }
 
-  public long nextBlockId(boolean isStriped) {
-    return blockIdManager.nextBlockId(isStriped);
+  public long nextBlockId(BlockType blockType) {
+    return blockIdManager.nextBlockId(blockType);
   }
 
   boolean isGenStampInFuture(Block block) {

+ 8 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.util.ReflectionUtils;
 
@@ -46,11 +47,13 @@ public class BlockPlacementPolicies{
     ecPolicy.initialize(conf, stats, clusterMap, host2datanodeMap);
   }
 
-  public BlockPlacementPolicy getPolicy(boolean isStriped){
-    if (isStriped) {
-      return ecPolicy;
-    } else {
-      return replicationPolicy;
+  public BlockPlacementPolicy getPolicy(BlockType blockType){
+    switch (blockType) {
+    case CONTIGUOUS: return replicationPolicy;
+    case STRIPED: return ecPolicy;
+    default:
+      throw new IllegalArgumentException(
+          "getPolicy received a BlockType that isn't supported.");
     }
   }
 }

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

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -60,16 +61,16 @@ public class BlockUnderConstructionFeature {
   private Block truncateBlock;
 
   public BlockUnderConstructionFeature(Block blk,
-      BlockUCState state, DatanodeStorageInfo[] targets, boolean isStriped) {
+      BlockUCState state, DatanodeStorageInfo[] targets, BlockType blockType) {
     assert getBlockUCState() != COMPLETE :
         "BlockUnderConstructionFeature cannot be in COMPLETE state";
     this.blockUCState = state;
-    setExpectedLocations(blk, targets, isStriped);
+    setExpectedLocations(blk, targets, blockType);
   }
 
   /** Set expected locations */
   public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets,
-      boolean isStriped) {
+      BlockType blockType) {
     if (targets == null) {
       return;
     }
@@ -86,7 +87,7 @@ public class BlockUnderConstructionFeature {
       if (targets[i] != null) {
         // when creating a new striped block we simply sequentially assign block
         // index to each storage
-        Block replicaBlock = isStriped ?
+        Block replicaBlock = blockType == BlockType.STRIPED ?
             new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) :
             block;
         replicas[offset++] = new ReplicaUnderConstruction(replicaBlock,

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

@@ -31,6 +31,7 @@ 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.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
@@ -227,7 +228,8 @@ final class FSDirTruncateOp {
 
     boolean shouldCopyOnTruncate = shouldCopyOnTruncate(fsn, file, oldBlock);
     if (newBlock == null) {
-      newBlock = (shouldCopyOnTruncate) ? fsn.createNewBlock(false)
+      newBlock = (shouldCopyOnTruncate) ?
+          fsn.createNewBlock(BlockType.CONTIGUOUS)
           : new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(),
           fsn.nextGenerationStamp(fsn.getBlockManager().isLegacyBlock(
               oldBlock)));

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

@@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -119,7 +120,7 @@ class FSDirWriteFileOp {
     FSNamesystem fsn = fsd.getFSNamesystem();
     final INodeFile file = fsn.checkLease(iip, holder, fileId);
     Preconditions.checkState(file.isUnderConstruction());
-    if (file.isStriped()) {
+    if (file.getBlockType() == BlockType.STRIPED) {
       return; // do not abandon block for striped file
     }
 
@@ -162,7 +163,7 @@ class FSDirWriteFileOp {
     final short numTargets;
     final byte storagePolicyID;
     String clientMachine;
-    final boolean isStriped;
+    final BlockType blockType;
 
     INodesInPath iip = fsn.dir.resolvePath(pc, src, fileId);
     FileState fileState = analyzeFileState(fsn, iip, fileId, clientName,
@@ -186,9 +187,9 @@ class FSDirWriteFileOp {
     blockSize = pendingFile.getPreferredBlockSize();
     clientMachine = pendingFile.getFileUnderConstructionFeature()
         .getClientMachine();
-    isStriped = pendingFile.isStriped();
+    blockType = pendingFile.getBlockType();
     ErasureCodingPolicy ecPolicy = null;
-    if (isStriped) {
+    if (blockType == BlockType.STRIPED) {
       ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(fsn, src);
       numTargets = (short) (ecPolicy.getSchema().getNumDataUnits()
           + ecPolicy.getSchema().getNumParityUnits());
@@ -197,7 +198,7 @@ class FSDirWriteFileOp {
     }
     storagePolicyID = pendingFile.getStoragePolicyID();
     return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID,
-                                      clientMachine, isStriped);
+                                      clientMachine, blockType);
   }
 
   static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk,
@@ -237,7 +238,7 @@ class FSDirWriteFileOp {
         // add new chosen targets to already allocated block and return
         BlockInfo lastBlockInFile = pendingFile.getLastBlock();
         lastBlockInFile.getUnderConstructionFeature().setExpectedLocations(
-            lastBlockInFile, targets, pendingFile.isStriped());
+            lastBlockInFile, targets, pendingFile.getBlockType());
         offset = pendingFile.computeFileSize();
         return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
       }
@@ -248,11 +249,11 @@ class FSDirWriteFileOp {
                                   ExtendedBlock.getLocalBlock(previous));
 
     // allocate new block, record block locations in INode.
-    final boolean isStriped = pendingFile.isStriped();
+    final BlockType blockType = pendingFile.getBlockType();
     // allocate new block, record block locations in INode.
-    Block newBlock = fsn.createNewBlock(isStriped);
+    Block newBlock = fsn.createNewBlock(blockType);
     INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
-    saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets, isStriped);
+    saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets, blockType);
 
     persistNewBlock(fsn, src, pendingFile);
     offset = pendingFile.computeFileSize();
@@ -282,7 +283,7 @@ class FSDirWriteFileOp {
     return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
                                     excludedNodesSet, r.blockSize,
                                     favoredNodesList, r.storagePolicyID,
-                                    r.isStriped, flags);
+                                    r.blockType, flags);
   }
 
   /**
@@ -420,14 +421,16 @@ class FSDirWriteFileOp {
       if (ecPolicy != null) {
         replication = ecPolicy.getId();
       }
+      final BlockType blockType = ecPolicy != null?
+          BlockType.STRIPED : BlockType.CONTIGUOUS;
       if (underConstruction) {
         newNode = newINodeFile(id, permissions, modificationTime,
             modificationTime, replication, preferredBlockSize, storagePolicyId,
-            ecPolicy != null);
+            blockType);
         newNode.toUnderConstruction(clientName, clientMachine);
       } else {
         newNode = newINodeFile(id, permissions, modificationTime, atime,
-            replication, preferredBlockSize, storagePolicyId, ecPolicy != null);
+            replication, preferredBlockSize, storagePolicyId, blockType);
       }
       newNode.setLocalName(localName);
       INodesInPath iip = fsd.addINode(existing, newNode,
@@ -459,7 +462,7 @@ class FSDirWriteFileOp {
    */
   private static BlockInfo addBlock(FSDirectory fsd, String path,
       INodesInPath inodesInPath, Block block, DatanodeStorageInfo[] targets,
-      boolean isStriped) throws IOException {
+      BlockType blockType) throws IOException {
     fsd.writeLock();
     try {
       final INodeFile fileINode = inodesInPath.getLastINode().asFile();
@@ -467,7 +470,7 @@ class FSDirWriteFileOp {
 
       // associate new last block for the file
       final BlockInfo blockInfo;
-      if (isStriped) {
+      if (blockType == BlockType.STRIPED) {
         ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
             fsd.getFSNamesystem(), inodesInPath);
         short numDataUnits = (short) ecPolicy.getNumDataUnits();
@@ -525,8 +528,10 @@ class FSDirWriteFileOp {
       if (ecPolicy != null) {
         replication = ecPolicy.getId();
       }
+      final BlockType blockType = ecPolicy != null?
+          BlockType.STRIPED : BlockType.CONTIGUOUS;
       INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
-          modTime, modTime, replication, preferredBlockSize, ecPolicy != null);
+          modTime, modTime, replication, preferredBlockSize, blockType);
       newNode.setLocalName(localName);
       newNode.toUnderConstruction(clientName, clientMachine);
       newiip = fsd.addINode(existing, newNode, permissions.getPermission());
@@ -698,17 +703,17 @@ class FSDirWriteFileOp {
   private static INodeFile newINodeFile(
       long id, PermissionStatus permissions, long mtime, long atime,
       short replication, long preferredBlockSize, byte storagePolicyId,
-      boolean isStriped) {
+      BlockType blockType) {
     return new INodeFile(id, null, permissions, mtime, atime,
         BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
-        storagePolicyId, isStriped);
+        storagePolicyId, blockType);
   }
 
   private static INodeFile newINodeFile(long id, PermissionStatus permissions,
       long mtime, long atime, short replication, long preferredBlockSize,
-      boolean isStriped) {
+      BlockType blockType) {
     return newINodeFile(id, permissions, mtime, atime, replication,
-        preferredBlockSize, (byte)0, isStriped);
+        preferredBlockSize, (byte)0, blockType);
   }
 
   /**
@@ -738,10 +743,10 @@ class FSDirWriteFileOp {
    */
   private static void saveAllocatedBlock(FSNamesystem fsn, String src,
       INodesInPath inodesInPath, Block newBlock, DatanodeStorageInfo[] targets,
-      boolean isStriped) throws IOException {
+      BlockType blockType) throws IOException {
     assert fsn.hasWriteLock();
     BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, targets,
-        isStriped);
+        blockType);
     logAllocatedBlock(src, b);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
   }
@@ -808,16 +813,16 @@ class FSDirWriteFileOp {
     final int numTargets;
     final byte storagePolicyID;
     final String clientMachine;
-    final boolean isStriped;
+    final BlockType blockType;
 
     ValidateAddBlockResult(
         long blockSize, int numTargets, byte storagePolicyID,
-        String clientMachine, boolean isStriped) {
+        String clientMachine, BlockType blockType) {
       this.blockSize = blockSize;
       this.numTargets = numTargets;
       this.storagePolicyID = storagePolicyID;
       this.clientMachine = clientMachine;
-      this.isStriped = isStriped;
+      this.blockType = blockType;
     }
   }
 }

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

@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -896,7 +897,8 @@ public class FSImageFormat {
       final long preferredBlockSize = in.readLong();
 
       return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
-          accessTime, replication, preferredBlockSize, (byte) 0, null, false);
+          accessTime, replication, preferredBlockSize, (byte) 0, null,
+          BlockType.CONTIGUOUS);
     }
 
     public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)

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

@@ -47,6 +47,7 @@ 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.BlockManager;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@@ -329,14 +330,14 @@ public final class FSImageFormatPBINode {
       INodeSection.INodeFile f = n.getFile();
       List<BlockProto> bp = f.getBlocksList();
       short replication = (short) f.getReplication();
-      boolean isStriped = f.getIsStriped();
+      BlockType blockType = PBHelperClient.convert(f.getBlockType());
       LoaderContext state = parent.getLoaderContext();
       ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
 
       BlockInfo[] blocks = new BlockInfo[bp.size()];
       for (int i = 0; i < bp.size(); ++i) {
         BlockProto b = bp.get(i);
-        if (isStriped) {
+        if (blockType == BlockType.STRIPED) {
           blocks[i] = new BlockInfoStriped(PBHelperClient.convert(b), ecPolicy);
         } else {
           blocks[i] = new BlockInfoContiguous(PBHelperClient.convert(b),
@@ -350,7 +351,7 @@ 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(), isStriped);
+          (byte)f.getStoragePolicyID(), blockType);
 
       if (f.hasAcl()) {
         int[] entries = AclEntryStatusFormat.toInt(loadAclEntries(
@@ -373,7 +374,7 @@ public final class FSImageFormatPBINode {
           BlockInfo lastBlk = file.getLastBlock();
           // replace the last block of file
           final BlockInfo ucBlk;
-          if (isStriped) {
+          if (blockType == BlockType.STRIPED) {
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
             ucBlk = new BlockInfoStriped(striped, ecPolicy);
           } else {
@@ -502,7 +503,7 @@ public final class FSImageFormatPBINode {
           .setPreferredBlockSize(file.getPreferredBlockSize())
           .setReplication(file.getFileReplication())
           .setStoragePolicyID(file.getLocalStoragePolicyID())
-          .setIsStriped(file.isStriped());
+          .setBlockType(PBHelperClient.convert(file.getBlockType()));
 
       AclFeature f = file.getAclFeature();
       if (f != null) {

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

@@ -207,6 +207,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -2541,7 +2542,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final long preferredblocksize;
     final byte storagePolicyID;
     final List<DatanodeStorageInfo> chosen;
-    final boolean isStriped;
+    final BlockType blockType;
     checkOperation(OperationCategory.READ);
     FSPermissionChecker pc = getPermissionChecker();
     readLock();
@@ -2558,7 +2559,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
       preferredblocksize = file.getPreferredBlockSize();
       storagePolicyID = file.getStoragePolicyID();
-      isStriped = file.isStriped();
+      blockType = file.getBlockType();
 
       //find datanode storages
       final DatanodeManager dm = blockManager.getDatanodeManager();
@@ -2576,7 +2577,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // choose new datanodes.
     final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode(
         src, numAdditionalNodes, clientnode, chosen, 
-        excludes, preferredblocksize, storagePolicyID, isStriped);
+        excludes, preferredblocksize, storagePolicyID, blockType);
     final LocatedBlock lb = BlockManager.newLocatedBlock(
         blk, targets, -1, false);
     blockManager.setBlockToken(lb, BlockTokenIdentifier.AccessMode.COPY);
@@ -2676,11 +2677,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * Create new block with a unique block id and a new generation stamp.
-   * @param isStriped is the file under striping or contiguous layout?
+   * @param blockType is the file under striping or contiguous layout?
    */
-  Block createNewBlock(boolean isStriped) throws IOException {
+  Block createNewBlock(BlockType blockType) throws IOException {
     assert hasWriteLock();
-    Block b = new Block(nextBlockId(isStriped), 0, 0);
+    Block b = new Block(nextBlockId(blockType), 0, 0);
     // Increment the generation stamp for every new block.
     b.setGenerationStamp(nextGenerationStamp(false));
     return b;
@@ -3194,7 +3195,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       // setup the last block locations from the blockManager if not known
       if (uc.getNumExpectedLocations() == 0) {
         uc.setExpectedLocations(lastBlock, blockManager.getStorages(lastBlock),
-            lastBlock.isStriped());
+            lastBlock.getBlockType());
       }
 
       if (uc.getNumExpectedLocations() == 0 && lastBlock.getNumBytes() == 0) {
@@ -4716,12 +4717,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * Increments, logs and then returns the block ID
-   * @param isStriped is the file under striping or contiguous layout?
+   * @param blockType is the file under striping or contiguous layout?
    */
-  private long nextBlockId(boolean isStriped) throws IOException {
+  private long nextBlockId(BlockType blockType) throws IOException {
     assert hasWriteLock();
     checkNameNodeSafeMode("Cannot get next block ID");
-    final long blockId = blockManager.nextBlockId(isStriped);
+    final long blockId = blockManager.nextBlockId(blockType);
     getEditLog().logAllocateBlockId(blockId);
     // NB: callers sync the log
     return blockId;
@@ -4936,7 +4937,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             "src=%s, oldBlock=%s, newBlock=%s, clientName=%s",
             src, oldBlock, newBlock, clientName);
     lastBlock.getUnderConstructionFeature().setExpectedLocations(lastBlock,
-        storages, lastBlock.isStriped());
+        storages, lastBlock.getBlockType());
 
     FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, logRetryCache);
   }

+ 40 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java

@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+import static org.apache.hadoop.hdfs.protocol.BlockType.CONTIGUOUS;
+import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 
@@ -43,6 +45,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
@@ -93,6 +96,10 @@ public class INodeFile extends INodeWithAdditionalFields
    * stores the EC policy ID, and in the future, we may further divide these
    * 11 bits to store both the EC policy ID and replication factor for erasure
    * coded blocks. The layout of this section is demonstrated as below.
+   *
+   * Another possible future extension is for future block types, in which case
+   * the 'Replica or EC' bit may be extended into the 11 bit field.
+   *
    * +---------------+-------------------------------+
    * |     1 bit     |             11 bit            |
    * +---------------+-------------------------------+
@@ -144,13 +151,29 @@ public class INodeFile extends INodeWithAdditionalFields
       return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
     }
 
+    // Union of all the block type masks. Currently there is only
+    // BLOCK_TYPE_MASK_STRIPED
+    static final long BLOCK_TYPE_MASK = 1 << 11;
+    // Mask to determine if the block type is striped.
+    static final long BLOCK_TYPE_MASK_STRIPED = 1 << 11;
+
     static boolean isStriped(long header) {
       long layoutRedundancy = BLOCK_LAYOUT_AND_REDUNDANCY.BITS.retrieve(header);
-      return (layoutRedundancy & (1 << 11)) != 0;
+      return (layoutRedundancy & BLOCK_TYPE_MASK) != 0;
+    }
+
+    static BlockType getBlockType(long header) {
+      long layoutRedundancy = BLOCK_LAYOUT_AND_REDUNDANCY.BITS.retrieve(header);
+      long blockType = layoutRedundancy & BLOCK_TYPE_MASK;
+      if (blockType == BLOCK_TYPE_MASK_STRIPED) {
+        return STRIPED;
+      } else {
+        return CONTIGUOUS;
+      }
     }
 
     static long toLong(long preferredBlockSize, short replication,
-        boolean isStriped, byte storagePolicyID) {
+        BlockType blockType, byte storagePolicyID) {
       Preconditions.checkArgument(replication >= 0 &&
           replication <= MAX_REDUNDANCY);
       long h = 0;
@@ -161,8 +184,8 @@ public class INodeFile extends INodeWithAdditionalFields
       // For erasure coded files, replication is used to store ec policy id
       // TODO: this is hacky. Add some utility to generate the layoutRedundancy
       long layoutRedundancy = 0;
-      if (isStriped) {
-        layoutRedundancy |= 1 << 11;
+      if (blockType == STRIPED) {
+        layoutRedundancy |= BLOCK_TYPE_MASK_STRIPED;
       }
       layoutRedundancy |= replication;
       h = BLOCK_LAYOUT_AND_REDUNDANCY.BITS.combine(layoutRedundancy, h);
@@ -180,18 +203,18 @@ public class INodeFile extends INodeWithAdditionalFields
             long atime, BlockInfo[] blklist, short replication,
             long preferredBlockSize) {
     this(id, name, permissions, mtime, atime, blklist, replication,
-        preferredBlockSize, (byte) 0, false);
+        preferredBlockSize, (byte) 0, CONTIGUOUS);
   }
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
       long atime, BlockInfo[] blklist, short replication,
-      long preferredBlockSize, byte storagePolicyID, boolean isStriped) {
+      long preferredBlockSize, byte storagePolicyID, BlockType blockType) {
     super(id, name, permissions, mtime, atime);
-    header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
+    header = HeaderFormat.toLong(preferredBlockSize, replication, blockType,
         storagePolicyID);
     if (blklist != null && blklist.length > 0) {
       for (BlockInfo b : blklist) {
-        Preconditions.checkArgument(b.isStriped() == isStriped);
+        Preconditions.checkArgument(b.getBlockType() == blockType);
       }
     }
     setBlocks(blklist);
@@ -554,6 +577,15 @@ public class INodeFile extends INodeWithAdditionalFields
     return HeaderFormat.isStriped(header);
   }
 
+  /**
+   * @return The type of the INodeFile based on block id.
+   */
+  @VisibleForTesting
+  @Override
+  public BlockType getBlockType() {
+    return HeaderFormat.getBlockType(header);
+  }
+
   @Override // INodeFileAttributes
   public long getHeaderLong() {
     return header;

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

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile.HeaderFormat;
 
 /**
@@ -32,6 +33,9 @@ public interface INodeFileAttributes extends INodeAttributes {
   /** @return whether the file is striped (instead of contiguous) */
   boolean isStriped();
 
+  /** @return whether the file is striped (instead of contiguous) */
+  BlockType getBlockType();
+
   /** @return the ID of the ErasureCodingPolicy */
   byte getErasureCodingPolicyID();
 
@@ -53,10 +57,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, boolean isStriped) {
+        byte storagePolicyID, XAttrFeature xAttrsFeature, BlockType blockType) {
       super(name, permissions, aclFeature, modificationTime, accessTime, 
           xAttrsFeature);
-      header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
+      header = HeaderFormat.toLong(preferredBlockSize, replication, blockType,
           storagePolicyID);
     }
 
@@ -80,6 +84,11 @@ public interface INodeFileAttributes extends INodeAttributes {
       return HeaderFormat.isStriped(header);
     }
 
+    @Override
+    public BlockType getBlockType() {
+      return HeaderFormat.getBlockType(header);
+    }
+
     @Override
     public byte getErasureCodingPolicyID() {
       if (isStriped()) {

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

@@ -721,7 +721,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
 
       // count mis replicated blocks
       BlockPlacementStatus blockPlacementStatus = bpPolicies.getPolicy(
-          lBlk.isStriped()).verifyBlockPlacement(lBlk.getLocations(),
+          lBlk.getBlockType()).verifyBlockPlacement(lBlk.getLocations(),
           targetFileReplication);
       if (!blockPlacementStatus.isPlacementPolicySatisfied()) {
         res.numMisReplicatedBlocks++;

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

@@ -237,7 +237,7 @@ public class FSImageFormatPBSnapshot {
               fileInPb.getAccessTime(), (short) fileInPb.getReplication(),
               fileInPb.getPreferredBlockSize(),
               (byte)fileInPb.getStoragePolicyID(), xAttrs,
-              fileInPb.getIsStriped());
+              PBHelperClient.convert(fileInPb.getBlockType()));
         }
 
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,

+ 14 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/OfflineImageReconstructor.java

@@ -649,8 +649,20 @@ class OfflineImageReconstructor {
     if (ival != null) {
       bld.setStoragePolicyID(ival);
     }
-    Boolean bval = node.removeChildBool(INODE_SECTION_IS_STRIPED);
-    bld.setIsStriped(bval);
+    String blockType = node.removeChildStr(INODE_SECTION_BLOCK_TYPE);
+    if(blockType != null) {
+      switch (blockType) {
+      case "CONTIGUOUS":
+        bld.setBlockType(HdfsProtos.BlockTypeProto.CONTIGUOUS);
+        break;
+      case "STRIPED":
+        bld.setBlockType(HdfsProtos.BlockTypeProto.STRIPED);
+        break;
+      default:
+        throw new IOException("INode XML found with unknown <blocktype> " +
+            blockType);
+      }
+    }
     inodeBld.setFile(bld);
     // Will check remaining keys and serialize in processINodeXml
   }

+ 6 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheD
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
@@ -130,7 +131,7 @@ public final class PBImageXmlWriter {
   public static final String INODE_SECTION_XATTRS = "xattrs";
   public static final String INODE_SECTION_STORAGE_POLICY_ID =
       "storagePolicyId";
-  public static final String INODE_SECTION_IS_STRIPED = "isStriped";
+  public static final String INODE_SECTION_BLOCK_TYPE = "blockType";
   public static final String INODE_SECTION_NS_QUOTA = "nsquota";
   public static final String INODE_SECTION_DS_QUOTA = "dsquota";
   public static final String INODE_SECTION_TYPE_QUOTA = "typeQuota";
@@ -492,8 +493,10 @@ public final class PBImageXmlWriter {
     if (f.hasStoragePolicyID()) {
       o(INODE_SECTION_STORAGE_POLICY_ID, f.getStoragePolicyID());
     }
-    if (f.getIsStriped()) {
-      out.print("<" + INODE_SECTION_IS_STRIPED + "/>");
+    if (f.getBlockType() != BlockTypeProto.CONTIGUOUS) {
+      out.print("<" + INODE_SECTION_BLOCK_TYPE + ">");
+      o(SECTION_NAME, f.getBlockType().name());
+      out.print("</" + INODE_SECTION_BLOCK_TYPE + ">\n");
     }
 
     if (f.hasFileUC()) {

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

@@ -128,7 +128,7 @@ message INodeSection {
   message XAttrFeatureProto {
     repeated XAttrCompactProto xAttrs = 1;
   }
-  
+
   message INodeFile {
     optional uint32 replication = 1;
     optional uint64 modificationTime = 2;
@@ -140,7 +140,7 @@ message INodeSection {
     optional AclFeatureProto acl = 8;
     optional XAttrFeatureProto xAttrs = 9;
     optional uint32 storagePolicyID = 10;
-    optional bool isStriped = 11;
+    optional BlockTypeProto blockType = 11;
   }
 
   message QuotaByStorageTypeEntryProto {

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
@@ -48,6 +49,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecover
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
@@ -199,6 +201,19 @@ public class TestPBHelper {
     assertEquals(b, b2);
   }
 
+  @Test
+  public void testConvertBlockType() {
+    BlockType bContiguous = BlockType.CONTIGUOUS;
+    BlockTypeProto bContiguousProto = PBHelperClient.convert(bContiguous);
+    BlockType bContiguous2 = PBHelperClient.convert(bContiguousProto);
+    assertEquals(bContiguous, bContiguous2);
+
+    BlockType bStriped = BlockType.STRIPED;
+    BlockTypeProto bStripedProto = PBHelperClient.convert(bStriped);
+    BlockType bStriped2 = PBHelperClient.convert(bStripedProto);
+    assertEquals(bStriped, bStriped2);
+  }
+
   private static BlockWithLocations getBlockWithLocations(
       int bid, boolean isStriped) {
     final String[] datanodeUuids = {"dn1", "dn2", "dn3"};

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

@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -167,7 +168,7 @@ public class TestDeadDatanode {
     // part of the cluster anymore
     DatanodeStorageInfo[] results = bm.chooseTarget4NewBlock("/hello", 3,
         clientNode, new HashSet<Node>(), 256 * 1024 * 1024L, null, (byte) 7,
-        false, null);
+        BlockType.CONTIGUOUS, null);
     for (DatanodeStorageInfo datanodeStorageInfo : results) {
       assertFalse("Dead node should not be choosen", datanodeStorageInfo
           .getDatanodeDescriptor().equals(clientNode));

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

@@ -33,9 +33,11 @@ import java.util.EnumSet;
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 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.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.Assert;
 
@@ -58,6 +60,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
@@ -160,7 +163,7 @@ public class TestFSImage {
     short replication = testECPolicy.getId();
     long preferredBlockSize = 128*1024*1024;
     INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
-        blks, replication, preferredBlockSize, (byte) 0, true);
+        blks, replication, preferredBlockSize, (byte) 0, BlockType.STRIPED);
     ByteArrayOutputStream bs = new ByteArrayOutputStream();
 
     //construct StripedBlocks for the INode
@@ -646,4 +649,14 @@ public class TestFSImage {
       }
     }
   }
+
+  @Test
+  public void testBlockTypeProtoDefaultsToContiguous() throws Exception {
+    INodeSection.INodeFile.Builder builder = INodeSection.INodeFile
+        .newBuilder();
+    INodeSection.INodeFile inodeFile = builder.build();
+    BlockType defaultBlockType = PBHelperClient.convert(inodeFile
+        .getBlockType());
+    assertEquals(defaultBlockType, BlockType.CONTIGUOUS);
+  }
 }

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

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.protocol.BlockType.CONTIGUOUS;
+import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertSame;
@@ -102,9 +104,15 @@ public class TestINodeFile {
         null, replication, preferredBlockSize);
   }
 
+  INodeFile createStripedINodeFile(short replication, long preferredBlockSize) {
+    return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
+        null, replication, preferredBlockSize,
+        HdfsConstants.WARM_STORAGE_POLICY_ID, STRIPED);
+  }
+
   private static INodeFile createINodeFile(byte storagePolicyID) {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-        null, (short)3, 1024L, storagePolicyID, false);
+        null, (short)3, 1024L, storagePolicyID, CONTIGUOUS);
   }
 
   @Test
@@ -215,14 +223,24 @@ public class TestINodeFile {
 
     dir.addChild(inf);
     assertEquals("d"+Path.SEPARATOR+"f", inf.getFullPathName());
-    
+
     root.addChild(dir);
     assertEquals(Path.SEPARATOR+"d"+Path.SEPARATOR+"f", inf.getFullPathName());
     assertEquals(Path.SEPARATOR+"d", dir.getFullPathName());
 
     assertEquals(Path.SEPARATOR, root.getFullPathName());
   }
-  
+
+  @Test
+  public void testGetBlockType() {
+    replication = 3;
+    preferredBlockSize = 128*1024*1024;
+    INodeFile inf = createINodeFile(replication, preferredBlockSize);
+    assertEquals(inf.getBlockType(), CONTIGUOUS);
+    INodeFile striped = createStripedINodeFile(replication, preferredBlockSize);
+    assertEquals(striped.getBlockType(), STRIPED);
+  }
+
   /**
    * FSDirectory#unprotectedSetQuota creates a new INodeDirectoryWithQuota to
    * replace the original INodeDirectory. Before HDFS-4243, the parent field of

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

@@ -46,6 +46,7 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.junit.Assert;
 import org.junit.Rule;
@@ -75,7 +76,8 @@ public class TestStripedINodeFile {
 
   private static INodeFile createStripedINodeFile() {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-        null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID, true);
+        null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID,
+        BlockType.STRIPED);
   }
 
   @Test