瀏覽代碼

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

Andrew Wang 8 年之前
父節點
當前提交
a2a5d7b5bc
共有 31 個文件被更改,包括 418 次插入109 次删除
  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() {
   public boolean isStriped() {
     return false;
     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.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveStats;
 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;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 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.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.ContentSummaryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CorruptFileBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.CryptoProtocolVersionProto;
 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());
     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[]) {
   static public DatanodeInfo[] convert(DatanodeInfoProto di[]) {
     if (di == null) return null;
     if (di == null) return null;
     DatanodeInfo[] result = new DatanodeInfo[di.length];
     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;
   RAM_DISK = 4;
 }
 }
 
 
+/**
+ * Types of recognized blocks.
+ */
+enum BlockTypeProto {
+  CONTIGUOUS = 0;
+  STRIPED = 1;
+}
+
 /**
 /**
  * A list of storage types. 
  * 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;
 package org.apache.hadoop.hdfs.server.balancer;
 
 
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.hadoop.hdfs.protocol.BlockType.CONTIGUOUS;
 
 
 import java.io.IOException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.io.PrintStream;
@@ -220,7 +221,7 @@ public class Balancer {
       ) throws UnsupportedActionException {
       ) throws UnsupportedActionException {
     BlockPlacementPolicies placementPolicies =
     BlockPlacementPolicies placementPolicies =
         new BlockPlacementPolicies(conf, null, null, null);
         new BlockPlacementPolicies(conf, null, null, null);
-    if (!(placementPolicies.getPolicy(false) instanceof
+    if (!(placementPolicies.getPolicy(CONTIGUOUS) instanceof
         BlockPlacementPolicyDefault)) {
         BlockPlacementPolicyDefault)) {
       throw new UnsupportedActionException(
       throw new UnsupportedActionException(
           "Balancer without BlockPlacementPolicyDefault");
           "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.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
 import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies;
 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;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations;
@@ -1274,7 +1275,7 @@ public class Dispatcher {
       }
       }
       datanodeInfos.add(target.getDatanodeInfo());
       datanodeInfos.add(target.getDatanodeInfo());
     }
     }
-    return placementPolicies.getPolicy(false).isMovable(
+    return placementPolicies.getPolicy(BlockType.CONTIGUOUS).isMovable(
         datanodeInfos, source.getDatanodeInfo(), target.getDatanodeInfo());
         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.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 
 import java.io.IOException;
 import java.io.IOException;
 
 
+import static org.apache.hadoop.hdfs.protocol.BlockType.STRIPED;
+
 /**
 /**
  * BlockIdManager allocates the generation stamps and the block ID. The
  * BlockIdManager allocates the generation stamps and the block ID. The
  * {@see FSNamesystem} is responsible for persisting the allocations in 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
    * 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) {
   boolean isGenStampInFuture(Block block) {
@@ -230,7 +238,7 @@ public class BlockIdManager {
   }
   }
 
 
   public static boolean isStripedBlockID(long id) {
   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 com.google.common.base.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.util.LightWeightGSet;
 import org.apache.hadoop.util.LightWeightGSet;
@@ -175,6 +176,8 @@ public abstract class BlockInfo extends Block
 
 
   public abstract boolean isStriped();
   public abstract boolean isStriped();
 
 
+  public abstract BlockType getBlockType();
+
   /** @return true if there is no datanode storage associated with the block */
   /** @return true if there is no datanode storage associated with the block */
   abstract boolean hasNoStorage();
   abstract boolean hasNoStorage();
 
 
@@ -262,11 +265,11 @@ public abstract class BlockInfo extends Block
       DatanodeStorageInfo[] targets) {
       DatanodeStorageInfo[] targets) {
     if (isComplete()) {
     if (isComplete()) {
       uc = new BlockUnderConstructionFeature(this, s, targets,
       uc = new BlockUnderConstructionFeature(this, s, targets,
-          this.isStriped());
+          this.getBlockType());
     } else {
     } else {
       // the block is already under construction
       // the block is already under construction
       uc.setBlockUCState(s);
       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.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.
  * Subclass of {@link BlockInfo}, used for a block with replication scheme.
@@ -92,6 +93,11 @@ public class BlockInfoContiguous extends BlockInfo {
     return false;
     return false;
   }
   }
 
 
+  @Override
+  public BlockType getBlockType() {
+    return BlockType.CONTIGUOUS;
+  }
+
   @Override
   @Override
   final boolean hasNoStorage() {
   final boolean hasNoStorage() {
     return getStorageInfo(0) == null;
     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.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
 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.BlockUCState;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.hdfs.util.StripedBlockUtil;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
@@ -208,6 +209,11 @@ public class BlockInfoStriped extends BlockInfo {
     return true;
     return true;
   }
   }
 
 
+  @Override
+  public BlockType getBlockType() {
+    return BlockType.STRIPED;
+  }
+
   @Override
   @Override
   public int numNodes() {
   public int numNodes() {
     assert this.storages != null : "BlockInfo is not initialized";
     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;
 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 static org.apache.hadoop.util.ExitUtil.terminate;
 
 
 import java.io.IOException;
 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.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
 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.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -642,7 +645,7 @@ public class BlockManager implements BlockStatsMXBean {
 
 
   @VisibleForTesting
   @VisibleForTesting
   public BlockPlacementPolicy getBlockPlacementPolicy() {
   public BlockPlacementPolicy getBlockPlacementPolicy() {
-    return placementPolicies.getPolicy(false);
+    return placementPolicies.getPolicy(CONTIGUOUS);
   }
   }
 
 
   /** Dump meta data to out. */
   /** Dump meta data to out. */
@@ -784,10 +787,13 @@ public class BlockManager implements BlockStatsMXBean {
   }
   }
 
 
   public int getDefaultStorageNum(BlockInfo block) {
   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) {
   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,
       // It is costly to extract the filename for which chooseTargets is called,
       // so for now we pass in the block collection itself.
       // so for now we pass in the block collection itself.
       final BlockPlacementPolicy placementPolicy =
       final BlockPlacementPolicy placementPolicy =
-          placementPolicies.getPolicy(rw.getBlock().isStriped());
+          placementPolicies.getPolicy(rw.getBlock().getBlockType());
       rw.chooseTargets(placementPolicy, storagePolicySuite, excludedNodes);
       rw.chooseTargets(placementPolicy, storagePolicySuite, excludedNodes);
     }
     }
 
 
@@ -1928,9 +1937,9 @@ public class BlockManager implements BlockStatsMXBean {
   /** Choose target for WebHDFS redirection. */
   /** Choose target for WebHDFS redirection. */
   public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src,
   public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src,
       DatanodeDescriptor clientnode, Set<Node> excludes, long blocksize) {
       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. */
   /** Choose target for getting additional datanodes for an existing pipeline. */
@@ -1941,9 +1950,11 @@ public class BlockManager implements BlockStatsMXBean {
       Set<Node> excludes,
       Set<Node> excludes,
       long blocksize,
       long blocksize,
       byte storagePolicyID,
       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,
     return blockplacement.chooseTarget(src, numAdditionalNodes, clientnode,
         chosen, true, excludes, blocksize, storagePolicy, null);
         chosen, true, excludes, blocksize, storagePolicy, null);
   }
   }
@@ -1962,12 +1973,14 @@ public class BlockManager implements BlockStatsMXBean {
       final long blocksize,
       final long blocksize,
       final List<String> favoredNodes,
       final List<String> favoredNodes,
       final byte storagePolicyID,
       final byte storagePolicyID,
-      final boolean isStriped,
+      final BlockType blockType,
       final EnumSet<AddBlockFlag> flags) throws IOException {
       final EnumSet<AddBlockFlag> flags) throws IOException {
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
     List<DatanodeDescriptor> favoredDatanodeDescriptors = 
         getDatanodeDescriptors(favoredNodes);
         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,
     final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src,
         numOfReplicas, client, excludedNodes, blocksize, 
         numOfReplicas, client, excludedNodes, blocksize, 
         favoredDatanodeDescriptors, storagePolicy, flags);
         favoredDatanodeDescriptors, storagePolicy, flags);
@@ -3428,7 +3441,7 @@ public class BlockManager implements BlockStatsMXBean {
       final Collection<DatanodeStorageInfo> nonExcess, BlockInfo storedBlock,
       final Collection<DatanodeStorageInfo> nonExcess, BlockInfo storedBlock,
       short replication, DatanodeDescriptor addedNode,
       short replication, DatanodeDescriptor addedNode,
       DatanodeDescriptor delNodeHint, List<StorageType> excessTypes) {
       DatanodeDescriptor delNodeHint, List<StorageType> excessTypes) {
-    BlockPlacementPolicy replicator = placementPolicies.getPolicy(false);
+    BlockPlacementPolicy replicator = placementPolicies.getPolicy(CONTIGUOUS);
     List<DatanodeStorageInfo> replicasToDelete = replicator
     List<DatanodeStorageInfo> replicasToDelete = replicator
         .chooseReplicasToDelete(nonExcess, nonExcess, replication, excessTypes,
         .chooseReplicasToDelete(nonExcess, nonExcess, replication, excessTypes,
             addedNode, delNodeHint);
             addedNode, delNodeHint);
@@ -3490,7 +3503,7 @@ public class BlockManager implements BlockStatsMXBean {
       return;
       return;
     }
     }
 
 
-    BlockPlacementPolicy placementPolicy = placementPolicies.getPolicy(true);
+    BlockPlacementPolicy placementPolicy = placementPolicies.getPolicy(STRIPED);
     // for each duplicated index, delete some replicas until only one left
     // for each duplicated index, delete some replicas until only one left
     for (int targetIndex = duplicated.nextSetBit(0); targetIndex >= 0;
     for (int targetIndex = duplicated.nextSetBit(0); targetIndex >= 0;
          targetIndex = duplicated.nextSetBit(targetIndex + 1)) {
          targetIndex = duplicated.nextSetBit(targetIndex + 1)) {
@@ -4145,9 +4158,10 @@ public class BlockManager implements BlockStatsMXBean {
       }
       }
     }
     }
     DatanodeInfo[] locs = liveNodes.toArray(new DatanodeInfo[liveNodes.size()]);
     DatanodeInfo[] locs = liveNodes.toArray(new DatanodeInfo[liveNodes.size()]);
+    BlockType blockType = storedBlock.getBlockType();
     BlockPlacementPolicy placementPolicy = placementPolicies
     BlockPlacementPolicy placementPolicy = placementPolicies
-        .getPolicy(storedBlock.isStriped());
-    int numReplicas = storedBlock.isStriped() ? ((BlockInfoStriped) storedBlock)
+        .getPolicy(blockType);
+    int numReplicas = blockType == STRIPED ? ((BlockInfoStriped) storedBlock)
         .getRealDataBlockNum() : storedBlock.getReplication();
         .getRealDataBlockNum() : storedBlock.getReplication();
     return placementPolicy.verifyBlockPlacement(locs, numReplicas)
     return placementPolicy.verifyBlockPlacement(locs, numReplicas)
         .isPlacementPolicySatisfied();
         .isPlacementPolicySatisfied();
@@ -4685,8 +4699,8 @@ public class BlockManager implements BlockStatsMXBean {
     return blockIdManager.isLegacyBlock(block);
     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) {
   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.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 
 
@@ -46,11 +47,13 @@ public class BlockPlacementPolicies{
     ecPolicy.initialize(conf, stats, clusterMap, host2datanodeMap);
     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;
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -60,16 +61,16 @@ public class BlockUnderConstructionFeature {
   private Block truncateBlock;
   private Block truncateBlock;
 
 
   public BlockUnderConstructionFeature(Block blk,
   public BlockUnderConstructionFeature(Block blk,
-      BlockUCState state, DatanodeStorageInfo[] targets, boolean isStriped) {
+      BlockUCState state, DatanodeStorageInfo[] targets, BlockType blockType) {
     assert getBlockUCState() != COMPLETE :
     assert getBlockUCState() != COMPLETE :
         "BlockUnderConstructionFeature cannot be in COMPLETE state";
         "BlockUnderConstructionFeature cannot be in COMPLETE state";
     this.blockUCState = state;
     this.blockUCState = state;
-    setExpectedLocations(blk, targets, isStriped);
+    setExpectedLocations(blk, targets, blockType);
   }
   }
 
 
   /** Set expected locations */
   /** Set expected locations */
   public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets,
   public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets,
-      boolean isStriped) {
+      BlockType blockType) {
     if (targets == null) {
     if (targets == null) {
       return;
       return;
     }
     }
@@ -86,7 +87,7 @@ public class BlockUnderConstructionFeature {
       if (targets[i] != null) {
       if (targets[i] != null) {
         // when creating a new striped block we simply sequentially assign block
         // when creating a new striped block we simply sequentially assign block
         // index to each storage
         // index to each storage
-        Block replicaBlock = isStriped ?
+        Block replicaBlock = blockType == BlockType.STRIPED ?
             new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) :
             new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) :
             block;
             block;
         replicas[offset++] = new ReplicaUnderConstruction(replicaBlock,
         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.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
@@ -227,7 +228,8 @@ final class FSDirTruncateOp {
 
 
     boolean shouldCopyOnTruncate = shouldCopyOnTruncate(fsn, file, oldBlock);
     boolean shouldCopyOnTruncate = shouldCopyOnTruncate(fsn, file, oldBlock);
     if (newBlock == null) {
     if (newBlock == null) {
-      newBlock = (shouldCopyOnTruncate) ? fsn.createNewBlock(false)
+      newBlock = (shouldCopyOnTruncate) ?
+          fsn.createNewBlock(BlockType.CONTIGUOUS)
           : new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(),
           : new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(),
           fsn.nextGenerationStamp(fsn.getBlockManager().isLegacyBlock(
           fsn.nextGenerationStamp(fsn.getBlockManager().isLegacyBlock(
               oldBlock)));
               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.BlockInfoStriped;
 
 
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 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.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -119,7 +120,7 @@ class FSDirWriteFileOp {
     FSNamesystem fsn = fsd.getFSNamesystem();
     FSNamesystem fsn = fsd.getFSNamesystem();
     final INodeFile file = fsn.checkLease(iip, holder, fileId);
     final INodeFile file = fsn.checkLease(iip, holder, fileId);
     Preconditions.checkState(file.isUnderConstruction());
     Preconditions.checkState(file.isUnderConstruction());
-    if (file.isStriped()) {
+    if (file.getBlockType() == BlockType.STRIPED) {
       return; // do not abandon block for striped file
       return; // do not abandon block for striped file
     }
     }
 
 
@@ -162,7 +163,7 @@ class FSDirWriteFileOp {
     final short numTargets;
     final short numTargets;
     final byte storagePolicyID;
     final byte storagePolicyID;
     String clientMachine;
     String clientMachine;
-    final boolean isStriped;
+    final BlockType blockType;
 
 
     INodesInPath iip = fsn.dir.resolvePath(pc, src, fileId);
     INodesInPath iip = fsn.dir.resolvePath(pc, src, fileId);
     FileState fileState = analyzeFileState(fsn, iip, fileId, clientName,
     FileState fileState = analyzeFileState(fsn, iip, fileId, clientName,
@@ -186,9 +187,9 @@ class FSDirWriteFileOp {
     blockSize = pendingFile.getPreferredBlockSize();
     blockSize = pendingFile.getPreferredBlockSize();
     clientMachine = pendingFile.getFileUnderConstructionFeature()
     clientMachine = pendingFile.getFileUnderConstructionFeature()
         .getClientMachine();
         .getClientMachine();
-    isStriped = pendingFile.isStriped();
+    blockType = pendingFile.getBlockType();
     ErasureCodingPolicy ecPolicy = null;
     ErasureCodingPolicy ecPolicy = null;
-    if (isStriped) {
+    if (blockType == BlockType.STRIPED) {
       ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(fsn, src);
       ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(fsn, src);
       numTargets = (short) (ecPolicy.getSchema().getNumDataUnits()
       numTargets = (short) (ecPolicy.getSchema().getNumDataUnits()
           + ecPolicy.getSchema().getNumParityUnits());
           + ecPolicy.getSchema().getNumParityUnits());
@@ -197,7 +198,7 @@ class FSDirWriteFileOp {
     }
     }
     storagePolicyID = pendingFile.getStoragePolicyID();
     storagePolicyID = pendingFile.getStoragePolicyID();
     return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID,
     return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID,
-                                      clientMachine, isStriped);
+                                      clientMachine, blockType);
   }
   }
 
 
   static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk,
   static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk,
@@ -237,7 +238,7 @@ class FSDirWriteFileOp {
         // add new chosen targets to already allocated block and return
         // add new chosen targets to already allocated block and return
         BlockInfo lastBlockInFile = pendingFile.getLastBlock();
         BlockInfo lastBlockInFile = pendingFile.getLastBlock();
         lastBlockInFile.getUnderConstructionFeature().setExpectedLocations(
         lastBlockInFile.getUnderConstructionFeature().setExpectedLocations(
-            lastBlockInFile, targets, pendingFile.isStriped());
+            lastBlockInFile, targets, pendingFile.getBlockType());
         offset = pendingFile.computeFileSize();
         offset = pendingFile.computeFileSize();
         return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
         return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
       }
       }
@@ -248,11 +249,11 @@ class FSDirWriteFileOp {
                                   ExtendedBlock.getLocalBlock(previous));
                                   ExtendedBlock.getLocalBlock(previous));
 
 
     // allocate new block, record block locations in INode.
     // 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.
     // allocate new block, record block locations in INode.
-    Block newBlock = fsn.createNewBlock(isStriped);
+    Block newBlock = fsn.createNewBlock(blockType);
     INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
     INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
-    saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets, isStriped);
+    saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets, blockType);
 
 
     persistNewBlock(fsn, src, pendingFile);
     persistNewBlock(fsn, src, pendingFile);
     offset = pendingFile.computeFileSize();
     offset = pendingFile.computeFileSize();
@@ -282,7 +283,7 @@ class FSDirWriteFileOp {
     return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
     return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode,
                                     excludedNodesSet, r.blockSize,
                                     excludedNodesSet, r.blockSize,
                                     favoredNodesList, r.storagePolicyID,
                                     favoredNodesList, r.storagePolicyID,
-                                    r.isStriped, flags);
+                                    r.blockType, flags);
   }
   }
 
 
   /**
   /**
@@ -420,14 +421,16 @@ class FSDirWriteFileOp {
       if (ecPolicy != null) {
       if (ecPolicy != null) {
         replication = ecPolicy.getId();
         replication = ecPolicy.getId();
       }
       }
+      final BlockType blockType = ecPolicy != null?
+          BlockType.STRIPED : BlockType.CONTIGUOUS;
       if (underConstruction) {
       if (underConstruction) {
         newNode = newINodeFile(id, permissions, modificationTime,
         newNode = newINodeFile(id, permissions, modificationTime,
             modificationTime, replication, preferredBlockSize, storagePolicyId,
             modificationTime, replication, preferredBlockSize, storagePolicyId,
-            ecPolicy != null);
+            blockType);
         newNode.toUnderConstruction(clientName, clientMachine);
         newNode.toUnderConstruction(clientName, clientMachine);
       } else {
       } else {
         newNode = newINodeFile(id, permissions, modificationTime, atime,
         newNode = newINodeFile(id, permissions, modificationTime, atime,
-            replication, preferredBlockSize, storagePolicyId, ecPolicy != null);
+            replication, preferredBlockSize, storagePolicyId, blockType);
       }
       }
       newNode.setLocalName(localName);
       newNode.setLocalName(localName);
       INodesInPath iip = fsd.addINode(existing, newNode,
       INodesInPath iip = fsd.addINode(existing, newNode,
@@ -459,7 +462,7 @@ class FSDirWriteFileOp {
    */
    */
   private static BlockInfo addBlock(FSDirectory fsd, String path,
   private static BlockInfo addBlock(FSDirectory fsd, String path,
       INodesInPath inodesInPath, Block block, DatanodeStorageInfo[] targets,
       INodesInPath inodesInPath, Block block, DatanodeStorageInfo[] targets,
-      boolean isStriped) throws IOException {
+      BlockType blockType) throws IOException {
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
       final INodeFile fileINode = inodesInPath.getLastINode().asFile();
       final INodeFile fileINode = inodesInPath.getLastINode().asFile();
@@ -467,7 +470,7 @@ class FSDirWriteFileOp {
 
 
       // associate new last block for the file
       // associate new last block for the file
       final BlockInfo blockInfo;
       final BlockInfo blockInfo;
-      if (isStriped) {
+      if (blockType == BlockType.STRIPED) {
         ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
         ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
             fsd.getFSNamesystem(), inodesInPath);
             fsd.getFSNamesystem(), inodesInPath);
         short numDataUnits = (short) ecPolicy.getNumDataUnits();
         short numDataUnits = (short) ecPolicy.getNumDataUnits();
@@ -525,8 +528,10 @@ class FSDirWriteFileOp {
       if (ecPolicy != null) {
       if (ecPolicy != null) {
         replication = ecPolicy.getId();
         replication = ecPolicy.getId();
       }
       }
+      final BlockType blockType = ecPolicy != null?
+          BlockType.STRIPED : BlockType.CONTIGUOUS;
       INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
       INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions,
-          modTime, modTime, replication, preferredBlockSize, ecPolicy != null);
+          modTime, modTime, replication, preferredBlockSize, blockType);
       newNode.setLocalName(localName);
       newNode.setLocalName(localName);
       newNode.toUnderConstruction(clientName, clientMachine);
       newNode.toUnderConstruction(clientName, clientMachine);
       newiip = fsd.addINode(existing, newNode, permissions.getPermission());
       newiip = fsd.addINode(existing, newNode, permissions.getPermission());
@@ -698,17 +703,17 @@ class FSDirWriteFileOp {
   private static INodeFile newINodeFile(
   private static INodeFile newINodeFile(
       long id, PermissionStatus permissions, long mtime, long atime,
       long id, PermissionStatus permissions, long mtime, long atime,
       short replication, long preferredBlockSize, byte storagePolicyId,
       short replication, long preferredBlockSize, byte storagePolicyId,
-      boolean isStriped) {
+      BlockType blockType) {
     return new INodeFile(id, null, permissions, mtime, atime,
     return new INodeFile(id, null, permissions, mtime, atime,
         BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
         BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize,
-        storagePolicyId, isStriped);
+        storagePolicyId, blockType);
   }
   }
 
 
   private static INodeFile newINodeFile(long id, PermissionStatus permissions,
   private static INodeFile newINodeFile(long id, PermissionStatus permissions,
       long mtime, long atime, short replication, long preferredBlockSize,
       long mtime, long atime, short replication, long preferredBlockSize,
-      boolean isStriped) {
+      BlockType blockType) {
     return newINodeFile(id, permissions, mtime, atime, replication,
     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,
   private static void saveAllocatedBlock(FSNamesystem fsn, String src,
       INodesInPath inodesInPath, Block newBlock, DatanodeStorageInfo[] targets,
       INodesInPath inodesInPath, Block newBlock, DatanodeStorageInfo[] targets,
-      boolean isStriped) throws IOException {
+      BlockType blockType) throws IOException {
     assert fsn.hasWriteLock();
     assert fsn.hasWriteLock();
     BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, targets,
     BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, targets,
-        isStriped);
+        blockType);
     logAllocatedBlock(src, b);
     logAllocatedBlock(src, b);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
   }
   }
@@ -808,16 +813,16 @@ class FSDirWriteFileOp {
     final int numTargets;
     final int numTargets;
     final byte storagePolicyID;
     final byte storagePolicyID;
     final String clientMachine;
     final String clientMachine;
-    final boolean isStriped;
+    final BlockType blockType;
 
 
     ValidateAddBlockResult(
     ValidateAddBlockResult(
         long blockSize, int numTargets, byte storagePolicyID,
         long blockSize, int numTargets, byte storagePolicyID,
-        String clientMachine, boolean isStriped) {
+        String clientMachine, BlockType blockType) {
       this.blockSize = blockSize;
       this.blockSize = blockSize;
       this.numTargets = numTargets;
       this.numTargets = numTargets;
       this.storagePolicyID = storagePolicyID;
       this.storagePolicyID = storagePolicyID;
       this.clientMachine = clientMachine;
       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.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -896,7 +897,8 @@ public class FSImageFormat {
       final long preferredBlockSize = in.readLong();
       final long preferredBlockSize = in.readLong();
 
 
       return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
       return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime,
-          accessTime, replication, preferredBlockSize, (byte) 0, null, false);
+          accessTime, replication, preferredBlockSize, (byte) 0, null,
+          BlockType.CONTIGUOUS);
     }
     }
 
 
     public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in)
     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.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 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;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@@ -329,14 +330,14 @@ public final class FSImageFormatPBINode {
       INodeSection.INodeFile f = n.getFile();
       INodeSection.INodeFile f = n.getFile();
       List<BlockProto> bp = f.getBlocksList();
       List<BlockProto> bp = f.getBlocksList();
       short replication = (short) f.getReplication();
       short replication = (short) f.getReplication();
-      boolean isStriped = f.getIsStriped();
+      BlockType blockType = PBHelperClient.convert(f.getBlockType());
       LoaderContext state = parent.getLoaderContext();
       LoaderContext state = parent.getLoaderContext();
       ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
       ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy();
 
 
       BlockInfo[] blocks = new BlockInfo[bp.size()];
       BlockInfo[] blocks = new BlockInfo[bp.size()];
       for (int i = 0; i < bp.size(); ++i) {
       for (int i = 0; i < bp.size(); ++i) {
         BlockProto b = bp.get(i);
         BlockProto b = bp.get(i);
-        if (isStriped) {
+        if (blockType == BlockType.STRIPED) {
           blocks[i] = new BlockInfoStriped(PBHelperClient.convert(b), ecPolicy);
           blocks[i] = new BlockInfoStriped(PBHelperClient.convert(b), ecPolicy);
         } else {
         } else {
           blocks[i] = new BlockInfoContiguous(PBHelperClient.convert(b),
           blocks[i] = new BlockInfoContiguous(PBHelperClient.convert(b),
@@ -350,7 +351,7 @@ public final class FSImageFormatPBINode {
       final INodeFile file = new INodeFile(n.getId(),
       final INodeFile file = new INodeFile(n.getId(),
           n.getName().toByteArray(), permissions, f.getModificationTime(),
           n.getName().toByteArray(), permissions, f.getModificationTime(),
           f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
           f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(),
-          (byte)f.getStoragePolicyID(), isStriped);
+          (byte)f.getStoragePolicyID(), blockType);
 
 
       if (f.hasAcl()) {
       if (f.hasAcl()) {
         int[] entries = AclEntryStatusFormat.toInt(loadAclEntries(
         int[] entries = AclEntryStatusFormat.toInt(loadAclEntries(
@@ -373,7 +374,7 @@ public final class FSImageFormatPBINode {
           BlockInfo lastBlk = file.getLastBlock();
           BlockInfo lastBlk = file.getLastBlock();
           // replace the last block of file
           // replace the last block of file
           final BlockInfo ucBlk;
           final BlockInfo ucBlk;
-          if (isStriped) {
+          if (blockType == BlockType.STRIPED) {
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
             BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
             ucBlk = new BlockInfoStriped(striped, ecPolicy);
             ucBlk = new BlockInfoStriped(striped, ecPolicy);
           } else {
           } else {
@@ -502,7 +503,7 @@ public final class FSImageFormatPBINode {
           .setPreferredBlockSize(file.getPreferredBlockSize())
           .setPreferredBlockSize(file.getPreferredBlockSize())
           .setReplication(file.getFileReplication())
           .setReplication(file.getFileReplication())
           .setStoragePolicyID(file.getLocalStoragePolicyID())
           .setStoragePolicyID(file.getLocalStoragePolicyID())
-          .setIsStriped(file.isStriped());
+          .setBlockType(PBHelperClient.convert(file.getBlockType()));
 
 
       AclFeature f = file.getAclFeature();
       AclFeature f = file.getAclFeature();
       if (f != null) {
       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.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 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.BlockUnderConstructionFeature;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -2541,7 +2542,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final long preferredblocksize;
     final long preferredblocksize;
     final byte storagePolicyID;
     final byte storagePolicyID;
     final List<DatanodeStorageInfo> chosen;
     final List<DatanodeStorageInfo> chosen;
-    final boolean isStriped;
+    final BlockType blockType;
     checkOperation(OperationCategory.READ);
     checkOperation(OperationCategory.READ);
     FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker pc = getPermissionChecker();
     readLock();
     readLock();
@@ -2558,7 +2559,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
       clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine);
       preferredblocksize = file.getPreferredBlockSize();
       preferredblocksize = file.getPreferredBlockSize();
       storagePolicyID = file.getStoragePolicyID();
       storagePolicyID = file.getStoragePolicyID();
-      isStriped = file.isStriped();
+      blockType = file.getBlockType();
 
 
       //find datanode storages
       //find datanode storages
       final DatanodeManager dm = blockManager.getDatanodeManager();
       final DatanodeManager dm = blockManager.getDatanodeManager();
@@ -2576,7 +2577,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     // choose new datanodes.
     // choose new datanodes.
     final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode(
     final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode(
         src, numAdditionalNodes, clientnode, chosen, 
         src, numAdditionalNodes, clientnode, chosen, 
-        excludes, preferredblocksize, storagePolicyID, isStriped);
+        excludes, preferredblocksize, storagePolicyID, blockType);
     final LocatedBlock lb = BlockManager.newLocatedBlock(
     final LocatedBlock lb = BlockManager.newLocatedBlock(
         blk, targets, -1, false);
         blk, targets, -1, false);
     blockManager.setBlockToken(lb, BlockTokenIdentifier.AccessMode.COPY);
     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.
    * 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();
     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.
     // Increment the generation stamp for every new block.
     b.setGenerationStamp(nextGenerationStamp(false));
     b.setGenerationStamp(nextGenerationStamp(false));
     return b;
     return b;
@@ -3194,7 +3195,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       // setup the last block locations from the blockManager if not known
       // setup the last block locations from the blockManager if not known
       if (uc.getNumExpectedLocations() == 0) {
       if (uc.getNumExpectedLocations() == 0) {
         uc.setExpectedLocations(lastBlock, blockManager.getStorages(lastBlock),
         uc.setExpectedLocations(lastBlock, blockManager.getStorages(lastBlock),
-            lastBlock.isStriped());
+            lastBlock.getBlockType());
       }
       }
 
 
       if (uc.getNumExpectedLocations() == 0 && lastBlock.getNumBytes() == 0) {
       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
    * 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();
     assert hasWriteLock();
     checkNameNodeSafeMode("Cannot get next block ID");
     checkNameNodeSafeMode("Cannot get next block ID");
-    final long blockId = blockManager.nextBlockId(isStriped);
+    final long blockId = blockManager.nextBlockId(blockType);
     getEditLog().logAllocateBlockId(blockId);
     getEditLog().logAllocateBlockId(blockId);
     // NB: callers sync the log
     // NB: callers sync the log
     return blockId;
     return blockId;
@@ -4936,7 +4937,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             "src=%s, oldBlock=%s, newBlock=%s, clientName=%s",
             "src=%s, oldBlock=%s, newBlock=%s, clientName=%s",
             src, oldBlock, newBlock, clientName);
             src, oldBlock, newBlock, clientName);
     lastBlock.getUnderConstructionFeature().setExpectedLocations(lastBlock,
     lastBlock.getUnderConstructionFeature().setExpectedLocations(lastBlock,
-        storages, lastBlock.isStriped());
+        storages, lastBlock.getBlockType());
 
 
     FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, logRetryCache);
     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;
 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.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.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_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.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 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
    * 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
    * 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.
    * 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            |
    * |     1 bit     |             11 bit            |
    * +---------------+-------------------------------+
    * +---------------+-------------------------------+
@@ -144,13 +151,29 @@ public class INodeFile extends INodeWithAdditionalFields
       return (byte)STORAGE_POLICY_ID.BITS.retrieve(header);
       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) {
     static boolean isStriped(long header) {
       long layoutRedundancy = BLOCK_LAYOUT_AND_REDUNDANCY.BITS.retrieve(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,
     static long toLong(long preferredBlockSize, short replication,
-        boolean isStriped, byte storagePolicyID) {
+        BlockType blockType, byte storagePolicyID) {
       Preconditions.checkArgument(replication >= 0 &&
       Preconditions.checkArgument(replication >= 0 &&
           replication <= MAX_REDUNDANCY);
           replication <= MAX_REDUNDANCY);
       long h = 0;
       long h = 0;
@@ -161,8 +184,8 @@ public class INodeFile extends INodeWithAdditionalFields
       // For erasure coded files, replication is used to store ec policy id
       // For erasure coded files, replication is used to store ec policy id
       // TODO: this is hacky. Add some utility to generate the layoutRedundancy
       // TODO: this is hacky. Add some utility to generate the layoutRedundancy
       long layoutRedundancy = 0;
       long layoutRedundancy = 0;
-      if (isStriped) {
-        layoutRedundancy |= 1 << 11;
+      if (blockType == STRIPED) {
+        layoutRedundancy |= BLOCK_TYPE_MASK_STRIPED;
       }
       }
       layoutRedundancy |= replication;
       layoutRedundancy |= replication;
       h = BLOCK_LAYOUT_AND_REDUNDANCY.BITS.combine(layoutRedundancy, h);
       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 atime, BlockInfo[] blklist, short replication,
             long preferredBlockSize) {
             long preferredBlockSize) {
     this(id, name, permissions, mtime, atime, blklist, replication,
     this(id, name, permissions, mtime, atime, blklist, replication,
-        preferredBlockSize, (byte) 0, false);
+        preferredBlockSize, (byte) 0, CONTIGUOUS);
   }
   }
 
 
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
   INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime,
       long atime, BlockInfo[] blklist, short replication,
       long atime, BlockInfo[] blklist, short replication,
-      long preferredBlockSize, byte storagePolicyID, boolean isStriped) {
+      long preferredBlockSize, byte storagePolicyID, BlockType blockType) {
     super(id, name, permissions, mtime, atime);
     super(id, name, permissions, mtime, atime);
-    header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
+    header = HeaderFormat.toLong(preferredBlockSize, replication, blockType,
         storagePolicyID);
         storagePolicyID);
     if (blklist != null && blklist.length > 0) {
     if (blklist != null && blklist.length > 0) {
       for (BlockInfo b : blklist) {
       for (BlockInfo b : blklist) {
-        Preconditions.checkArgument(b.isStriped() == isStriped);
+        Preconditions.checkArgument(b.getBlockType() == blockType);
       }
       }
     }
     }
     setBlocks(blklist);
     setBlocks(blklist);
@@ -554,6 +577,15 @@ public class INodeFile extends INodeWithAdditionalFields
     return HeaderFormat.isStriped(header);
     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
   @Override // INodeFileAttributes
   public long getHeaderLong() {
   public long getHeaderLong() {
     return header;
     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.classification.InterfaceAudience;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile.HeaderFormat;
 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) */
   /** @return whether the file is striped (instead of contiguous) */
   boolean isStriped();
   boolean isStriped();
 
 
+  /** @return whether the file is striped (instead of contiguous) */
+  BlockType getBlockType();
+
   /** @return the ID of the ErasureCodingPolicy */
   /** @return the ID of the ErasureCodingPolicy */
   byte getErasureCodingPolicyID();
   byte getErasureCodingPolicyID();
 
 
@@ -53,10 +57,10 @@ public interface INodeFileAttributes extends INodeAttributes {
     public SnapshotCopy(byte[] name, PermissionStatus permissions,
     public SnapshotCopy(byte[] name, PermissionStatus permissions,
         AclFeature aclFeature, long modificationTime, long accessTime,
         AclFeature aclFeature, long modificationTime, long accessTime,
         short replication, long preferredBlockSize,
         short replication, long preferredBlockSize,
-        byte storagePolicyID, XAttrFeature xAttrsFeature, boolean isStriped) {
+        byte storagePolicyID, XAttrFeature xAttrsFeature, BlockType blockType) {
       super(name, permissions, aclFeature, modificationTime, accessTime, 
       super(name, permissions, aclFeature, modificationTime, accessTime, 
           xAttrsFeature);
           xAttrsFeature);
-      header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped,
+      header = HeaderFormat.toLong(preferredBlockSize, replication, blockType,
           storagePolicyID);
           storagePolicyID);
     }
     }
 
 
@@ -80,6 +84,11 @@ public interface INodeFileAttributes extends INodeAttributes {
       return HeaderFormat.isStriped(header);
       return HeaderFormat.isStriped(header);
     }
     }
 
 
+    @Override
+    public BlockType getBlockType() {
+      return HeaderFormat.getBlockType(header);
+    }
+
     @Override
     @Override
     public byte getErasureCodingPolicyID() {
     public byte getErasureCodingPolicyID() {
       if (isStriped()) {
       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
       // count mis replicated blocks
       BlockPlacementStatus blockPlacementStatus = bpPolicies.getPolicy(
       BlockPlacementStatus blockPlacementStatus = bpPolicies.getPolicy(
-          lBlk.isStriped()).verifyBlockPlacement(lBlk.getLocations(),
+          lBlk.getBlockType()).verifyBlockPlacement(lBlk.getLocations(),
           targetFileReplication);
           targetFileReplication);
       if (!blockPlacementStatus.isPlacementPolicySatisfied()) {
       if (!blockPlacementStatus.isPlacementPolicySatisfied()) {
         res.numMisReplicatedBlocks++;
         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.getAccessTime(), (short) fileInPb.getReplication(),
               fileInPb.getPreferredBlockSize(),
               fileInPb.getPreferredBlockSize(),
               (byte)fileInPb.getStoragePolicyID(), xAttrs,
               (byte)fileInPb.getStoragePolicyID(), xAttrs,
-              fileInPb.getIsStriped());
+              PBHelperClient.convert(fileInPb.getBlockType()));
         }
         }
 
 
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
         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) {
     if (ival != null) {
       bld.setStoragePolicyID(ival);
       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);
     inodeBld.setFile(bld);
     // Will check remaining keys and serialize in processINodeXml
     // 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.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
 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.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatPBINode;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SectionName;
 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_XATTRS = "xattrs";
   public static final String INODE_SECTION_STORAGE_POLICY_ID =
   public static final String INODE_SECTION_STORAGE_POLICY_ID =
       "storagePolicyId";
       "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_NS_QUOTA = "nsquota";
   public static final String INODE_SECTION_DS_QUOTA = "dsquota";
   public static final String INODE_SECTION_DS_QUOTA = "dsquota";
   public static final String INODE_SECTION_TYPE_QUOTA = "typeQuota";
   public static final String INODE_SECTION_TYPE_QUOTA = "typeQuota";
@@ -492,8 +493,10 @@ public final class PBImageXmlWriter {
     if (f.hasStoragePolicyID()) {
     if (f.hasStoragePolicyID()) {
       o(INODE_SECTION_STORAGE_POLICY_ID, f.getStoragePolicyID());
       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()) {
     if (f.hasFileUC()) {

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

@@ -128,7 +128,7 @@ message INodeSection {
   message XAttrFeatureProto {
   message XAttrFeatureProto {
     repeated XAttrCompactProto xAttrs = 1;
     repeated XAttrCompactProto xAttrs = 1;
   }
   }
-  
+
   message INodeFile {
   message INodeFile {
     optional uint32 replication = 1;
     optional uint32 replication = 1;
     optional uint64 modificationTime = 2;
     optional uint64 modificationTime = 2;
@@ -140,7 +140,7 @@ message INodeSection {
     optional AclFeatureProto acl = 8;
     optional AclFeatureProto acl = 8;
     optional XAttrFeatureProto xAttrs = 9;
     optional XAttrFeatureProto xAttrs = 9;
     optional uint32 storagePolicyID = 10;
     optional uint32 storagePolicyID = 10;
-    optional bool isStriped = 11;
+    optional BlockTypeProto blockType = 11;
   }
   }
 
 
   message QuotaByStorageTypeEntryProto {
   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.fs.StorageType;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.DatanodeInfoBuilder;
 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.DatanodeProtocolProtos.DatanodeRegistrationProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 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.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.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
@@ -199,6 +201,19 @@ public class TestPBHelper {
     assertEquals(b, b2);
     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(
   private static BlockWithLocations getBlockWithLocations(
       int bid, boolean isStriped) {
       int bid, boolean isStriped) {
     final String[] datanodeUuids = {"dn1", "dn2", "dn3"};
     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.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 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.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -167,7 +168,7 @@ public class TestDeadDatanode {
     // part of the cluster anymore
     // part of the cluster anymore
     DatanodeStorageInfo[] results = bm.chooseTarget4NewBlock("/hello", 3,
     DatanodeStorageInfo[] results = bm.chooseTarget4NewBlock("/hello", 3,
         clientNode, new HashSet<Node>(), 256 * 1024 * 1024L, null, (byte) 7,
         clientNode, new HashSet<Node>(), 256 * 1024 * 1024L, null, (byte) 7,
-        false, null);
+        BlockType.CONTIGUOUS, null);
     for (DatanodeStorageInfo datanodeStorageInfo : results) {
     for (DatanodeStorageInfo datanodeStorageInfo : results) {
       assertFalse("Dead node should not be choosen", datanodeStorageInfo
       assertFalse("Dead node should not be choosen", datanodeStorageInfo
           .getDatanodeDescriptor().equals(clientNode));
           .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.Block;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 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.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.protocol.BlockType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.Assert;
 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.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 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.NNStorage.NameNodeDirType;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.hadoop.test.PathUtils;
@@ -160,7 +163,7 @@ public class TestFSImage {
     short replication = testECPolicy.getId();
     short replication = testECPolicy.getId();
     long preferredBlockSize = 128*1024*1024;
     long preferredBlockSize = 128*1024*1024;
     INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
     INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
-        blks, replication, preferredBlockSize, (byte) 0, true);
+        blks, replication, preferredBlockSize, (byte) 0, BlockType.STRIPED);
     ByteArrayOutputStream bs = new ByteArrayOutputStream();
     ByteArrayOutputStream bs = new ByteArrayOutputStream();
 
 
     //construct StripedBlocks for the INode
     //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;
 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.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertSame;
@@ -102,9 +104,15 @@ public class TestINodeFile {
         null, replication, preferredBlockSize);
         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) {
   private static INodeFile createINodeFile(byte storagePolicyID) {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-        null, (short)3, 1024L, storagePolicyID, false);
+        null, (short)3, 1024L, storagePolicyID, CONTIGUOUS);
   }
   }
 
 
   @Test
   @Test
@@ -215,14 +223,24 @@ public class TestINodeFile {
 
 
     dir.addChild(inf);
     dir.addChild(inf);
     assertEquals("d"+Path.SEPARATOR+"f", inf.getFullPathName());
     assertEquals("d"+Path.SEPARATOR+"f", inf.getFullPathName());
-    
+
     root.addChild(dir);
     root.addChild(dir);
     assertEquals(Path.SEPARATOR+"d"+Path.SEPARATOR+"f", inf.getFullPathName());
     assertEquals(Path.SEPARATOR+"d"+Path.SEPARATOR+"f", inf.getFullPathName());
     assertEquals(Path.SEPARATOR+"d", dir.getFullPathName());
     assertEquals(Path.SEPARATOR+"d", dir.getFullPathName());
 
 
     assertEquals(Path.SEPARATOR, root.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
    * FSDirectory#unprotectedSetQuota creates a new INodeDirectoryWithQuota to
    * replace the original INodeDirectory. Before HDFS-4243, the parent field of
    * 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.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 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.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Rule;
@@ -75,7 +76,8 @@ public class TestStripedINodeFile {
 
 
   private static INodeFile createStripedINodeFile() {
   private static INodeFile createStripedINodeFile() {
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
     return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
-        null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID, true);
+        null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID,
+        BlockType.STRIPED);
   }
   }
 
 
   @Test
   @Test