You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wa...@apache.org on 2015/09/30 17:42:02 UTC

[02/50] [abbrv] hadoop git commit: Merge commit '456e901a4c5c639267ee87b8e5f1319f256d20c2' (HDFS-6407. Add sorting and pagination in the datanode tab of the NN Web UI. Contributed by Haohui Mai.) into HDFS-7285-merge

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 2a9ab39,a5053bc..bae033b
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@@ -32,31 -29,23 +32,33 @@@ import org.apache.hadoop.security.Acces
  @InterfaceAudience.Private
  public interface Namesystem extends RwLock, SafeMode {
    /** Is this name system running? */
-   public boolean isRunning();
+   boolean isRunning();
  
    /** Check if the user has superuser privilege. */
-   public void checkSuperuserPrivilege() throws AccessControlException;
+   void checkSuperuserPrivilege() throws AccessControlException;
  
    /** @return the block pool ID */
-   public String getBlockPoolId();
+   String getBlockPoolId();
  
-   public boolean isInStandbyState();
+   boolean isInStandbyState();
  
-   public boolean isGenStampInFuture(Block block);
+   boolean isGenStampInFuture(Block block);
  
-   public void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
+   void adjustSafeModeBlockTotals(int deltaSafe, int deltaTotal);
  
-   public void checkOperation(OperationCategory read) throws StandbyException;
+   void checkOperation(OperationCategory read) throws StandbyException;
  
-   public boolean isInSnapshot(BlockCollection bc);
 -  boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC);
++  boolean isInSnapshot(BlockCollection bc);
 +
 +  /**
 +   * Gets the ECZone for path
 +   * @param src
 +   *          - path
 +   * @return {@link ErasureCodingZone}
 +   * @throws IOException
 +   */
-   public ErasureCodingZone getErasureCodingZoneForPath(String src)
++  ErasureCodingZone getErasureCodingZoneForPath(String src)
 +      throws IOException;
+ 
+   CacheManager getCacheManager();
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
index 0bcb860,64ad1f6..0162f85
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
@@@ -240,16 -240,13 +241,15 @@@ public class FSImageFormatPBSnapshot 
          FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
              pbf.getFileSize());
          List<BlockProto> bpl = pbf.getBlocksList();
 +        // in file diff there can only be contiguous blocks
-         BlockInfoContiguous[] blocks = new BlockInfoContiguous[bpl.size()];
+         BlockInfo[] blocks = new BlockInfo[bpl.size()];
          for(int j = 0, e = bpl.size(); j < e; ++j) {
            Block blk = PBHelper.convert(bpl.get(j));
-           BlockInfoContiguous storedBlock =
-               (BlockInfoContiguous) fsn.getBlockManager().getStoredBlock(blk);
+           BlockInfo storedBlock =  fsn.getBlockManager().getStoredBlock(blk);
            if(storedBlock == null) {
 -            storedBlock = fsn.getBlockManager().addBlockCollection(
 -                new BlockInfoContiguous(blk, copy.getFileReplication()), file);
 +            storedBlock = (BlockInfoContiguous) fsn.getBlockManager()
 +                .addBlockCollectionWithCheck(new BlockInfoContiguous(blk,
 +                    copy.getFileReplication()), file);
            }
            blocks[j] = storedBlock;
          }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index f44ea76,6b8388e..442e127
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@@ -22,8 -22,7 +22,8 @@@ import java.util.List
  
  import org.apache.hadoop.hdfs.protocol.Block;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
  import org.apache.hadoop.hdfs.server.namenode.INode;
  import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
index bb59a17,9f8aef5..e944b81
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
@@@ -103,7 -109,100 +109,101 @@@ public class TestLayoutVersion 
      assertEquals(LAST_COMMON_FEATURE.getInfo().getLayoutVersion() - 1,
          first.getInfo().getLayoutVersion());
    }
-   
+ 
+   /**
+    * Tests expected values for minimum compatible layout version in NameNode
+    * features.  TRUNCATE, APPEND_NEW_BLOCK and QUOTA_BY_STORAGE_TYPE are all
+    * features that launched in the same release.  TRUNCATE was added first, so
+    * we expect all 3 features to have a minimum compatible layout version equal
+    * to TRUNCATE's layout version.  All features older than that existed prior
+    * to the concept of a minimum compatible layout version, so for each one, the
+    * minimum compatible layout version must be equal to itself.
+    */
+   @Test
+   public void testNameNodeFeatureMinimumCompatibleLayoutVersions() {
+     int baseLV = NameNodeLayoutVersion.Feature.TRUNCATE.getInfo()
+         .getLayoutVersion();
+     EnumSet<NameNodeLayoutVersion.Feature> compatibleFeatures = EnumSet.of(
+         NameNodeLayoutVersion.Feature.TRUNCATE,
+         NameNodeLayoutVersion.Feature.APPEND_NEW_BLOCK,
 -        NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE);
++        NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE,
++        NameNodeLayoutVersion.Feature.ERASURE_CODING);
+     for (LayoutFeature f : compatibleFeatures) {
+       assertEquals(String.format("Expected minimum compatible layout version " +
+           "%d for feature %s.", baseLV, f), baseLV,
+           f.getInfo().getMinimumCompatibleLayoutVersion());
+     }
+     List<LayoutFeature> features = new ArrayList<>();
+     features.addAll(EnumSet.allOf(LayoutVersion.Feature.class));
+     features.addAll(EnumSet.allOf(NameNodeLayoutVersion.Feature.class));
+     for (LayoutFeature f : features) {
+       if (!compatibleFeatures.contains(f)) {
+         assertEquals(String.format("Expected feature %s to have minimum " +
+             "compatible layout version set to itself.", f),
+             f.getInfo().getLayoutVersion(),
+             f.getInfo().getMinimumCompatibleLayoutVersion());
+       }
+     }
+   }
+ 
+   /**
+    * Tests that NameNode features are listed in order of minimum compatible
+    * layout version.  It would be inconsistent to have features listed out of
+    * order with respect to minimum compatible layout version, because it would
+    * imply going back in time to change compatibility logic in a software release
+    * that had already shipped.
+    */
+   @Test
+   public void testNameNodeFeatureMinimumCompatibleLayoutVersionAscending() {
+     LayoutFeature prevF = null;
+     for (LayoutFeature f : EnumSet.allOf(NameNodeLayoutVersion.Feature.class)) {
+       if (prevF != null) {
+         assertTrue(String.format("Features %s and %s not listed in order of " +
+             "minimum compatible layout version.", prevF, f),
+             f.getInfo().getMinimumCompatibleLayoutVersion() <=
+             prevF.getInfo().getMinimumCompatibleLayoutVersion());
+       } else {
+         prevF = f;
+       }
+     }
+   }
+ 
+   /**
+    * Tests that attempting to add a new NameNode feature out of order with
+    * respect to minimum compatible layout version will fail fast.
+    */
+   @Test(expected=AssertionError.class)
+   public void testNameNodeFeatureMinimumCompatibleLayoutVersionOutOfOrder() {
+     FeatureInfo ancestorF = LayoutVersion.Feature.RESERVED_REL2_4_0.getInfo();
+     LayoutFeature f = mock(LayoutFeature.class);
+     when(f.getInfo()).thenReturn(new FeatureInfo(
+         ancestorF.getLayoutVersion() - 1, ancestorF.getLayoutVersion(),
+         ancestorF.getMinimumCompatibleLayoutVersion() + 1, "Invalid feature.",
+         false));
+     Map<Integer, SortedSet<LayoutFeature>> features = new HashMap<>();
+     LayoutVersion.updateMap(features, LayoutVersion.Feature.values());
+     LayoutVersion.updateMap(features, new LayoutFeature[] { f });
+   }
+ 
+   /**
+    * Asserts the current minimum compatible layout version of the software, if a
+    * release were created from the codebase right now.  This test is meant to
+    * make developers stop and reconsider if they introduce a change that requires
+    * a new minimum compatible layout version.  This would make downgrade
+    * impossible.
+    */
+   @Test
+   public void testCurrentMinimumCompatibleLayoutVersion() {
+     int expectedMinCompatLV = NameNodeLayoutVersion.Feature.TRUNCATE.getInfo()
+         .getLayoutVersion();
+     int actualMinCompatLV = LayoutVersion.getMinimumCompatibleLayoutVersion(
+         NameNodeLayoutVersion.Feature.values());
+     assertEquals("The minimum compatible layout version has changed.  " +
+         "Downgrade to prior versions is no longer possible.  Please either " +
+         "restore compatibility, or if the incompatibility is intentional, " +
+         "then update this assertion.", expectedMinCompatLV, actualMinCompatLV);
+   }
+ 
    /**
     * Given feature {@code f}, ensures the layout version of that feature
     * supports all the features supported by it's ancestor.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 99b460a,f8f4329..ad8f204
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@@ -59,8 -66,13 +66,14 @@@ import org.apache.hadoop.hdfs.Distribut
  import org.apache.hadoop.hdfs.HdfsConfiguration;
  import org.apache.hadoop.hdfs.MiniDFSCluster;
  import org.apache.hadoop.hdfs.NameNodeProxies;
- import org.apache.hadoop.hdfs.protocol.*;
+ import org.apache.hadoop.hdfs.protocol.Block;
+ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
++import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
  import org.apache.hadoop.hdfs.server.balancer.Balancer.Cli;
  import org.apache.hadoop.hdfs.server.balancer.Balancer.Parameters;
  import org.apache.hadoop.hdfs.server.balancer.Balancer.Result;
@@@ -130,26 -142,11 +144,28 @@@ public class TestBalancer 
      conf.setLong(DFS_HEARTBEAT_INTERVAL_KEY, 1);
      conf.setInt(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500);
      conf.setInt(DFS_DATANODE_LAZY_WRITER_INTERVAL_SEC, 1);
-     conf.setInt(DFS_DATANODE_RAM_DISK_LOW_WATERMARK_BYTES, DEFAULT_RAM_DISK_BLOCK_SIZE);
      LazyPersistTestCase.initCacheManipulator();
+ 
+     conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
    }
  
 +  int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
 +  int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
 +  int groupSize = dataBlocks + parityBlocks;
 +  private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 +  private final static int stripesPerBlock = 4;
 +  static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock;
 +
 +  static void initConfWithStripe(Configuration conf) {
 +    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE);
 +    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
 +    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
 +    SimulatedFSDataset.setFactory(conf);
 +    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L);
 +    conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L);
++    conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L);
 +  }
 +
    /* create a file with a length of <code>fileLen</code> */
    static void createFile(MiniDFSCluster cluster, Path filePath, long fileLen,
        short replicationFactor, int nnIndex)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 3e28236,6553185..dcb852b
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@@ -1255,13 -1246,13 +1255,13 @@@ public class TestReplicationPolicy 
      when(dn.isDecommissioned()).thenReturn(true);
      when(storage.getState()).thenReturn(DatanodeStorage.State.NORMAL);
      when(storage.getDatanodeDescriptor()).thenReturn(dn);
-     when(storage.removeBlock(any(BlockInfoContiguous.class))).thenReturn(true);
-     when(storage.addBlock(any(BlockInfoContiguous.class))).thenReturn
+     when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true);
+     when(storage.addBlock(any(BlockInfo.class))).thenReturn
          (DatanodeStorageInfo.AddBlockResult.ADDED);
 -    ucBlock.addStorage(storage);
 +    ucBlock.addStorage(storage, ucBlock);
  
 -    when(mbc.setLastBlock((BlockInfo) any(), (DatanodeStorageInfo[]) any()))
 -    .thenReturn(ucBlock);
 +    BlockInfo lastBlk = mbc.getLastBlock();
 +    when(mbc.getLastBlock()).thenReturn(lastBlk, ucBlock);
  
      bm.convertLastBlockToUnderConstruction(mbc, 0L);
  

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
index 261d397,d3d814c..2598fa8
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
@@@ -35,12 -34,7 +35,11 @@@ import org.apache.hadoop.hdfs.DFSUtil
  import org.apache.hadoop.hdfs.HdfsConfiguration;
  import org.apache.hadoop.hdfs.MiniDFSCluster;
  import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 +import org.apache.hadoop.hdfs.NameNodeProxies;
 +import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 +import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 +import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
- import org.apache.hadoop.hdfs.server.balancer.Dispatcher;
  import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DBlock;
  import org.apache.hadoop.hdfs.server.balancer.ExitStatus;
  import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector;
@@@ -368,120 -409,4 +414,119 @@@ public class TestMover 
        cluster.shutdown();
      }
    }
 +
 +  int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
 +  int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
 +  private final static int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 +  private final static int stripesPerBlock = 4;
 +  static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock;
 +
 +  static void initConfWithStripe(Configuration conf) {
 +    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE);
 +    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
 +    conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L);
 +    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false);
-     Dispatcher.setBlockMoveWaitTime(3000L);
 +  }
 +
 +  @Test(timeout = 300000)
 +  public void testMoverWithStripedFile() throws Exception {
 +    final Configuration conf = new HdfsConfiguration();
 +    initConfWithStripe(conf);
 +
 +    // start 10 datanodes
 +    int numOfDatanodes =10;
 +    int storagesPerDatanode=2;
 +    long capacity = 10 * DEFAULT_STRIPE_BLOCK_SIZE;
 +    long[][] capacities = new long[numOfDatanodes][storagesPerDatanode];
 +    for (int i = 0; i < numOfDatanodes; i++) {
 +      for(int j=0;j<storagesPerDatanode;j++){
 +        capacities[i][j]=capacity;
 +      }
 +    }
 +    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
 +        .numDataNodes(numOfDatanodes)
 +        .storagesPerDatanode(storagesPerDatanode)
 +        .storageTypes(new StorageType[][]{
 +            {StorageType.DISK, StorageType.DISK},
 +            {StorageType.DISK, StorageType.DISK},
 +            {StorageType.DISK, StorageType.DISK},
 +            {StorageType.DISK, StorageType.DISK},
 +            {StorageType.DISK, StorageType.DISK},
 +            {StorageType.DISK, StorageType.ARCHIVE},
 +            {StorageType.DISK, StorageType.ARCHIVE},
 +            {StorageType.DISK, StorageType.ARCHIVE},
 +            {StorageType.DISK, StorageType.ARCHIVE},
 +            {StorageType.DISK, StorageType.ARCHIVE}})
 +        .storageCapacities(capacities)
 +        .build();
 +
 +    try {
 +      cluster.waitActive();
 +
 +      // set "/bar" directory with HOT storage policy.
 +      ClientProtocol client = NameNodeProxies.createProxy(conf,
 +          cluster.getFileSystem(0).getUri(), ClientProtocol.class).getProxy();
 +      String barDir = "/bar";
 +      client.mkdirs(barDir, new FsPermission((short) 777), true);
 +      client.setStoragePolicy(barDir,
-           HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
++          HdfsConstants.HOT_STORAGE_POLICY_NAME);
 +      // set "/bar" directory with EC zone.
 +      client.createErasureCodingZone(barDir, null);
 +
 +      // write file to barDir
 +      final String fooFile = "/bar/foo";
 +      long fileLen = 20 * DEFAULT_STRIPE_BLOCK_SIZE ;
 +      DFSTestUtil.createFile(cluster.getFileSystem(), new Path(fooFile),
 +          fileLen,(short) 3, 0);
 +
 +      // verify storage types and locations
 +      LocatedBlocks locatedBlocks =
 +          client.getBlockLocations(fooFile, 0, fileLen);
 +      for(LocatedBlock lb : locatedBlocks.getLocatedBlocks()){
 +        for( StorageType type : lb.getStorageTypes()){
 +          Assert.assertEquals(StorageType.DISK, type);
 +        }
 +      }
 +      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
 +          dataBlocks + parityBlocks);
 +
 +      // start 5 more datanodes
 +      numOfDatanodes +=5;
 +      capacities = new long[5][storagesPerDatanode];
 +      for (int i = 0; i < 5; i++) {
 +        for(int j=0;j<storagesPerDatanode;j++){
 +          capacities[i][j]=capacity;
 +        }
 +      }
 +      cluster.startDataNodes(conf, 5,
 +          new StorageType[][]{
 +              {StorageType.ARCHIVE, StorageType.ARCHIVE},
 +              {StorageType.ARCHIVE, StorageType.ARCHIVE},
 +              {StorageType.ARCHIVE, StorageType.ARCHIVE},
 +              {StorageType.ARCHIVE, StorageType.ARCHIVE},
 +              {StorageType.ARCHIVE, StorageType.ARCHIVE}},
 +          true, null, null, null,capacities, null, false, false, false, null);
 +      cluster.triggerHeartbeats();
 +
 +      // move file to ARCHIVE
 +      client.setStoragePolicy(barDir, "COLD");
 +      // run Mover
 +      int rc = ToolRunner.run(conf, new Mover.Cli(),
 +          new String[] { "-p", barDir });
 +      Assert.assertEquals("Movement to ARCHIVE should be successfull", 0, rc);
 +
 +      // verify storage types and locations
 +      locatedBlocks = client.getBlockLocations(fooFile, 0, fileLen);
 +      for(LocatedBlock lb : locatedBlocks.getLocatedBlocks()){
 +        for( StorageType type : lb.getStorageTypes()){
 +          Assert.assertEquals(StorageType.ARCHIVE, type);
 +        }
 +      }
 +      DFSTestUtil.verifyLocatedStripedBlocks(locatedBlocks,
 +          dataBlocks + parityBlocks);
 +
 +    }finally{
 +      cluster.shutdown();
 +    }
 +  }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index 4ae10db,0000000..fa92c64
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@@ -1,433 -1,0 +1,433 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hdfs.server.namenode;
 +
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.hdfs.DFSStripedOutputStream;
 +import org.apache.hadoop.hdfs.DFSTestUtil;
 +import org.apache.hadoop.hdfs.DistributedFileSystem;
 +import org.apache.hadoop.hdfs.HdfsConfiguration;
 +import org.apache.hadoop.hdfs.MiniDFSCluster;
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 +import org.apache.hadoop.hdfs.protocol.DatanodeID;
 +import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 +import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 +import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 +import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
++import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 +import org.apache.hadoop.hdfs.server.datanode.DataNode;
 +import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
 +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
 +import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 +import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 +import org.apache.hadoop.io.IOUtils;
 +import org.junit.After;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.Test;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 +import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 +import static org.junit.Assert.assertEquals;
 +
 +public class TestAddStripedBlocks {
 +  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
 +      HdfsConstants.NUM_PARITY_BLOCKS;
 +
 +  private MiniDFSCluster cluster;
 +  private DistributedFileSystem dfs;
 +
 +  @Before
 +  public void setup() throws IOException {
 +    cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
 +        .numDataNodes(GROUP_SIZE).build();
 +    cluster.waitActive();
 +    dfs = cluster.getFileSystem();
 +    dfs.getClient().createErasureCodingZone("/", null);
 +  }
 +
 +  @After
 +  public void tearDown() {
 +    if (cluster != null) {
 +      cluster.shutdown();
 +    }
 +  }
 +
 +  /**
 +   * Make sure the IDs of striped blocks do not conflict
 +   */
 +  @Test
 +  public void testAllocateBlockId() throws Exception {
 +    Path testPath = new Path("/testfile");
 +    // create a file while allocates a new block
 +    DFSTestUtil.writeFile(dfs, testPath, "hello, world!");
 +    LocatedBlocks lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0);
 +    final long firstId = lb.get(0).getBlock().getBlockId();
 +    // delete the file
 +    dfs.delete(testPath, true);
 +
 +    // allocate a new block, and make sure the new block's id does not conflict
 +    // with the previous one
 +    DFSTestUtil.writeFile(dfs, testPath, "hello again");
 +    lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0);
 +    final long secondId = lb.get(0).getBlock().getBlockId();
 +    Assert.assertEquals(firstId + HdfsServerConstants.MAX_BLOCKS_IN_GROUP, secondId);
 +  }
 +
 +  private static void writeAndFlushStripedOutputStream(
 +      DFSStripedOutputStream out, int chunkSize) throws IOException {
 +    // FSOutputSummer.BUFFER_NUM_CHUNKS == 9
 +    byte[] toWrite = new byte[chunkSize * 9 + 1];
 +    out.write(toWrite);
 +    DFSTestUtil.flushInternal(out);
 +  }
 +
 +  @Test (timeout=60000)
 +  public void testAddStripedBlock() throws Exception {
 +    final Path file = new Path("/file1");
 +    // create an empty file
 +    FSDataOutputStream out = null;
 +    try {
 +      out = dfs.create(file, (short) 1);
 +      writeAndFlushStripedOutputStream(
 +          (DFSStripedOutputStream) out.getWrappedStream(),
 +          DFS_BYTES_PER_CHECKSUM_DEFAULT);
 +
 +      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
 +      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
 +
 +      BlockInfo[] blocks = fileNode.getBlocks();
 +      assertEquals(1, blocks.length);
 +      Assert.assertTrue(blocks[0].isStriped());
 +
 +      checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), true);
 +
 +      // restart NameNode to check editlog
 +      cluster.restartNameNode(true);
 +      fsdir = cluster.getNamesystem().getFSDirectory();
 +      fileNode = fsdir.getINode4Write(file.toString()).asFile();
 +      blocks = fileNode.getBlocks();
 +      assertEquals(1, blocks.length);
 +      Assert.assertTrue(blocks[0].isStriped());
 +      checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
 +
 +      // save namespace, restart namenode, and check
 +      dfs = cluster.getFileSystem();
 +      dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
 +      dfs.saveNamespace();
 +      dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
 +      cluster.restartNameNode(true);
 +      fsdir = cluster.getNamesystem().getFSDirectory();
 +      fileNode = fsdir.getINode4Write(file.toString()).asFile();
 +      blocks = fileNode.getBlocks();
 +      assertEquals(1, blocks.length);
 +      Assert.assertTrue(blocks[0].isStriped());
 +      checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
 +    } finally {
 +      IOUtils.cleanup(null, out);
 +    }
 +  }
 +
 +  private void checkStripedBlockUC(BlockInfoStriped block,
 +      boolean checkReplica) {
 +    assertEquals(0, block.numNodes());
 +    Assert.assertFalse(block.isComplete());
 +    Assert.assertEquals(HdfsConstants.NUM_DATA_BLOCKS, block.getDataBlockNum());
 +    Assert.assertEquals(HdfsConstants.NUM_PARITY_BLOCKS,
 +        block.getParityBlockNum());
 +    Assert.assertEquals(0,
 +        block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK);
 +
-     final BlockInfoUnderConstructionStriped blockUC =
-         (BlockInfoUnderConstructionStriped) block;
++    final BlockInfoStripedUnderConstruction blockUC =
++        (BlockInfoStripedUnderConstruction) block;
 +    Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
 +        blockUC.getBlockUCState());
 +    if (checkReplica) {
 +      Assert.assertEquals(GROUP_SIZE, blockUC.getNumExpectedLocations());
 +      DatanodeStorageInfo[] storages = blockUC.getExpectedStorageLocations();
 +      for (DataNode dn : cluster.getDataNodes()) {
 +        Assert.assertTrue(includeDataNode(dn.getDatanodeId(), storages));
 +      }
 +    }
 +  }
 +
 +  private boolean includeDataNode(DatanodeID dn, DatanodeStorageInfo[] storages) {
 +    for (DatanodeStorageInfo storage : storages) {
 +      if (storage.getDatanodeDescriptor().equals(dn)) {
 +        return true;
 +      }
 +    }
 +    return false;
 +  }
 +
 +  @Test
 +  public void testGetLocatedStripedBlocks() throws Exception {
 +    final Path file = new Path("/file1");
 +    // create an empty file
 +    FSDataOutputStream out = null;
 +    try {
 +      out = dfs.create(file, (short) 1);
 +      writeAndFlushStripedOutputStream(
 +          (DFSStripedOutputStream) out.getWrappedStream(),
 +          DFS_BYTES_PER_CHECKSUM_DEFAULT);
 +
 +      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
 +      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
-       BlockInfoUnderConstructionStriped lastBlk =
-           (BlockInfoUnderConstructionStriped) fileNode.getLastBlock();
++      BlockInfoStripedUnderConstruction lastBlk =
++          (BlockInfoStripedUnderConstruction) fileNode.getLastBlock();
 +      DatanodeInfo[] expectedDNs = DatanodeStorageInfo
 +          .toDatanodeInfos(lastBlk.getExpectedStorageLocations());
 +      int[] indices = lastBlk.getBlockIndices();
 +
 +      LocatedBlocks blks = dfs.getClient().getLocatedBlocks(file.toString(), 0L);
 +      Assert.assertEquals(1, blks.locatedBlockCount());
 +      LocatedBlock lblk = blks.get(0);
 +
 +      Assert.assertTrue(lblk instanceof LocatedStripedBlock);
 +      DatanodeInfo[] datanodes = lblk.getLocations();
 +      int[] blockIndices = ((LocatedStripedBlock) lblk).getBlockIndices();
 +      Assert.assertEquals(GROUP_SIZE, datanodes.length);
 +      Assert.assertEquals(GROUP_SIZE, blockIndices.length);
 +      Assert.assertArrayEquals(indices, blockIndices);
 +      Assert.assertArrayEquals(expectedDNs, datanodes);
 +    } finally {
 +      IOUtils.cleanup(null, out);
 +    }
 +  }
 +
 +  /**
-    * Test BlockInfoUnderConstructionStriped#addReplicaIfNotPresent in different
++   * Test BlockInfoStripedUnderConstruction#addReplicaIfNotPresent in different
 +   * scenarios.
 +   */
 +  @Test
 +  public void testAddUCReplica() throws Exception {
 +    final Path file = new Path("/file1");
 +    final List<String> storageIDs = new ArrayList<>();
 +    // create an empty file
 +    FSDataOutputStream out = null;
 +    try {
 +      out = dfs.create(file, (short) 1);
 +
 +      // 1. create the UC striped block
 +      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
 +      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
 +      cluster.getNamesystem().getAdditionalBlock(file.toString(),
 +          fileNode.getId(), dfs.getClient().getClientName(), null, null, null);
 +      BlockInfo lastBlock = fileNode.getLastBlock();
-       BlockInfoUnderConstructionStriped ucBlock =
-           (BlockInfoUnderConstructionStriped) lastBlock;
++      BlockInfoStripedUnderConstruction ucBlock =
++          (BlockInfoStripedUnderConstruction) lastBlock;
 +
 +      DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
 +      int[] indices = ucBlock.getBlockIndices();
 +      Assert.assertEquals(GROUP_SIZE, locs.length);
 +      Assert.assertEquals(GROUP_SIZE, indices.length);
 +
 +      // 2. mimic incremental block reports and make sure the uc-replica list in
 +      // the BlockInfoUCStriped is correct
 +      int i = 0;
 +      for (DataNode dn : cluster.getDataNodes()) {
 +        final Block block = new Block(lastBlock.getBlockId() + i++,
 +            0, lastBlock.getGenerationStamp());
 +        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
 +        storageIDs.add(storage.getStorageID());
 +        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
 +            .makeReportForReceivedBlock(block, BlockStatus.RECEIVING_BLOCK,
 +                storage);
 +        for (StorageReceivedDeletedBlocks report : reports) {
 +          cluster.getNamesystem().processIncrementalBlockReport(
 +              dn.getDatanodeId(), report);
 +        }
 +      }
 +
 +      // make sure lastBlock is correct and the storages have been updated
 +      locs = ucBlock.getExpectedStorageLocations();
 +      indices = ucBlock.getBlockIndices();
 +      Assert.assertEquals(GROUP_SIZE, locs.length);
 +      Assert.assertEquals(GROUP_SIZE, indices.length);
 +      for (DatanodeStorageInfo newstorage : locs) {
 +        Assert.assertTrue(storageIDs.contains(newstorage.getStorageID()));
 +      }
 +    } finally {
 +      IOUtils.cleanup(null, out);
 +    }
 +
 +    // 3. restart the namenode. mimic the full block reports and check the
 +    // uc-replica list again
 +    cluster.restartNameNode(true);
 +    final String bpId = cluster.getNamesystem().getBlockPoolId();
 +    INodeFile fileNode = cluster.getNamesystem().getFSDirectory()
 +        .getINode4Write(file.toString()).asFile();
 +    BlockInfo lastBlock = fileNode.getLastBlock();
 +    int i = GROUP_SIZE - 1;
 +    for (DataNode dn : cluster.getDataNodes()) {
 +      String storageID = storageIDs.get(i);
 +      final Block block = new Block(lastBlock.getBlockId() + i--,
 +          lastBlock.getGenerationStamp(), 0);
 +      DatanodeStorage storage = new DatanodeStorage(storageID);
 +      List<ReplicaBeingWritten> blocks = new ArrayList<>();
 +      ReplicaBeingWritten replica = new ReplicaBeingWritten(block, null, null,
 +          null);
 +      blocks.add(replica);
 +      BlockListAsLongs bll = BlockListAsLongs.encode(blocks);
 +      StorageBlockReport[] reports = {new StorageBlockReport(storage,
 +          bll)};
 +      cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId),
 +          bpId, reports, null);
 +    }
 +
-     BlockInfoUnderConstructionStriped ucBlock =
-         (BlockInfoUnderConstructionStriped) lastBlock;
++    BlockInfoStripedUnderConstruction ucBlock =
++        (BlockInfoStripedUnderConstruction) lastBlock;
 +    DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
 +    int[] indices = ucBlock.getBlockIndices();
 +    Assert.assertEquals(GROUP_SIZE, locs.length);
 +    Assert.assertEquals(GROUP_SIZE, indices.length);
 +    for (i = 0; i < GROUP_SIZE; i++) {
 +      Assert.assertEquals(storageIDs.get(i),
 +          locs[GROUP_SIZE - 1 - i].getStorageID());
 +      Assert.assertEquals(GROUP_SIZE - i - 1, indices[i]);
 +    }
 +  }
 +
 +  @Test
 +  public void testCheckStripedReplicaCorrupt() throws Exception {
 +    final int numBlocks = 4;
 +    final int numStripes = 4;
 +    final Path filePath = new Path("/corrupt");
 +    final FSNamesystem ns = cluster.getNameNode().getNamesystem();
 +    final BlockManager bm = ns.getBlockManager();
 +    DFSTestUtil.createStripedFile(cluster, filePath, null,
 +        numBlocks, numStripes, false);
 +
 +    INodeFile fileNode = ns.getFSDirectory().getINode(filePath.toString()).
 +        asFile();
 +    Assert.assertTrue(fileNode.isStriped());
 +    BlockInfo stored = fileNode.getBlocks()[0];
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(0, ns.getCorruptReplicaBlocks());
 +
 +    // Now send a block report with correct size
 +    DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
 +    final Block reported = new Block(stored);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
 +    StorageReceivedDeletedBlocks[] reports = DFSTestUtil
 +        .makeReportForReceivedBlock(reported,
 +            ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(0).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(0, ns.getCorruptReplicaBlocks());
 +
 +    // Now send a block report with wrong size
 +    reported.setBlockId(stored.getBlockId() + 1);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE - 1);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +            ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(1).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +
 +    // Now send a parity block report with correct size
 +    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(2).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +
 +    // Now send a parity block report with wrong size
 +    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 1);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(3).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    // the total number of corrupted block info is still 1
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +    // 2 internal blocks corrupted
 +    Assert.assertEquals(2, bm.getCorruptReplicas(stored).size());
 +
 +    // Now change the size of stored block, and test verifying the last
 +    // block size
 +    stored.setNumBytes(stored.getNumBytes() + 10);
 +    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS + 2);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(4).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
 +
 +    // Now send a parity block report with correct size based on adjusted
 +    // size of stored block
 +    /** Now stored block has {@link numStripes} full stripes + a cell + 10 */
 +    stored.setNumBytes(stored.getNumBytes() + BLOCK_STRIPED_CELL_SIZE);
 +    reported.setBlockId(stored.getBlockId());
 +    reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(0).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
 +
 +    reported.setBlockId(stored.getBlockId() + 1);
 +    reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 10);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(5).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
 +
 +    reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS);
 +    reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE);
 +    reports = DFSTestUtil.makeReportForReceivedBlock(reported,
 +        ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
 +    ns.processIncrementalBlockReport(
 +        cluster.getDataNodes().get(2).getDatanodeId(), reports[0]);
 +    BlockManagerTestUtil.updateState(ns.getBlockManager());
 +    Assert.assertEquals(1, ns.getCorruptReplicaBlocks());
 +    Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
index 92c329e,c5262d4..ec3d924
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
@@@ -126,4 -132,40 +132,41 @@@ public class TestDeadDatanode 
      assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
          .getAction());
    }
+ 
+   @Test
+   public void testDeadNodeAsBlockTarget() throws Exception {
+     Configuration conf = new HdfsConfiguration();
+     conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, 500);
+     conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
+     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
+     cluster.waitActive();
+ 
+     String poolId = cluster.getNamesystem().getBlockPoolId();
+     // wait for datanode to be marked live
+     DataNode dn = cluster.getDataNodes().get(0);
+     DatanodeRegistration reg = DataNodeTestUtils.getDNRegistrationForBP(cluster
+         .getDataNodes().get(0), poolId);
+     // Get the updated datanode descriptor
+     BlockManager bm = cluster.getNamesystem().getBlockManager();
+     DatanodeManager dm = bm.getDatanodeManager();
+     Node clientNode = dm.getDatanode(reg);
+ 
+     DFSTestUtil.waitForDatanodeState(cluster, reg.getDatanodeUuid(), true,
+         20000);
+ 
+     // Shutdown and wait for datanode to be marked dead
+     dn.shutdown();
+     DFSTestUtil.waitForDatanodeState(cluster, reg.getDatanodeUuid(), false,
+         20000);
+     // Get the updated datanode descriptor available in DNM
+     // choose the targets, but local node should not get selected as this is not
+     // part of the cluster anymore
+     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);
+     for (DatanodeStorageInfo datanodeStorageInfo : results) {
+       assertFalse("Dead node should not be choosen", datanodeStorageInfo
+           .getDatanodeDescriptor().equals(clientNode));
+     }
+   }
  }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index d1c7600,df20fd6..d3689f3
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@@ -17,13 -17,11 +17,17 @@@
   */
  package org.apache.hadoop.hdfs.server.namenode;
  
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertFalse;
+ import static org.junit.Assert.assertTrue;
+ 
  import java.io.File;
 +import java.io.DataOutput;
 +import java.io.DataOutputStream;
 +import java.io.DataInput;
 +import java.io.DataInputStream;
 +import java.io.ByteArrayOutputStream;
 +import java.io.ByteArrayInputStream;
  import java.io.IOException;
  import java.util.EnumSet;
  
@@@ -51,19 -40,16 +55,16 @@@ import org.apache.hadoop.hdfs.MiniDFSCl
  import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 -import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
  import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
+ import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
  import org.apache.hadoop.hdfs.util.MD5FileUtils;
  import org.apache.hadoop.test.GenericTestUtils;
  import org.apache.hadoop.test.PathUtils;
  import org.junit.Test;
  
 +import static org.junit.Assert.assertArrayEquals;
- import static org.junit.Assert.assertEquals;
- import static org.junit.Assert.assertNotNull;
- import static org.junit.Assert.assertTrue;
- import static org.junit.Assert.assertFalse;
 +
  public class TestFSImage {
  
    private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ =
@@@ -137,123 -120,45 +138,162 @@@
      }
    }
  
 +  private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf,
 +                                               boolean isUC) throws IOException{
 +    // contruct a INode with StripedBlock for saving and loading
 +    fsn.createErasureCodingZone("/", null, false);
 +    long id = 123456789;
 +    byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes();
 +    PermissionStatus permissionStatus = new PermissionStatus("testuser_a",
 +            "testuser_groups", new FsPermission((short)0x755));
 +    long mtime = 1426222916-3600;
 +    long atime = 1426222916;
 +    BlockInfoContiguous[] blks = new BlockInfoContiguous[0];
 +    short replication = 3;
 +    long preferredBlockSize = 128*1024*1024;
 +    INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
 +        blks, replication, preferredBlockSize, (byte) 0, true);
 +    ByteArrayOutputStream bs = new ByteArrayOutputStream();
 +
 +    //construct StripedBlocks for the INode
 +    BlockInfoStriped[] stripedBlks = new BlockInfoStriped[3];
 +    long stripedBlkId = 10000001;
 +    long timestamp = mtime+3600;
 +    for (int i = 0; i < stripedBlks.length; i++) {
 +      stripedBlks[i] = new BlockInfoStriped(
 +              new Block(stripedBlkId + i, preferredBlockSize, timestamp),
 +              testECPolicy);
 +      file.addBlock(stripedBlks[i]);
 +    }
 +
 +    final String client = "testClient";
 +    final String clientMachine = "testClientMachine";
 +    final String path = "testUnderConstructionPath";
 +
 +    //save the INode to byte array
 +    DataOutput out = new DataOutputStream(bs);
 +    if (isUC) {
 +      file.toUnderConstruction(client, clientMachine);
 +      FSImageSerialization.writeINodeUnderConstruction((DataOutputStream) out,
 +          file, path);
 +    } else {
 +      FSImageSerialization.writeINodeFile(file, out, false);
 +    }
 +    DataInput in = new DataInputStream(
 +            new ByteArrayInputStream(bs.toByteArray()));
 +
 +    // load the INode from the byte array
 +    INodeFile fileByLoaded;
 +    if (isUC) {
 +      fileByLoaded = FSImageSerialization.readINodeUnderConstruction(in,
 +              fsn, fsn.getFSImage().getLayoutVersion());
 +    } else {
 +      fileByLoaded = (INodeFile) new FSImageFormat.Loader(conf, fsn)
 +              .loadINodeWithLocalName(false, in, false);
 +    }
 +
 +    assertEquals(id, fileByLoaded.getId() );
 +    assertArrayEquals(isUC ? path.getBytes() : name,
 +        fileByLoaded.getLocalName().getBytes());
 +    assertEquals(permissionStatus.getUserName(),
 +        fileByLoaded.getPermissionStatus().getUserName());
 +    assertEquals(permissionStatus.getGroupName(),
 +        fileByLoaded.getPermissionStatus().getGroupName());
 +    assertEquals(permissionStatus.getPermission(),
 +        fileByLoaded.getPermissionStatus().getPermission());
 +    assertEquals(mtime, fileByLoaded.getModificationTime());
 +    assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime());
 +    // TODO for striped blocks, we currently save and load them as contiguous
 +    // blocks to/from legacy fsimage
 +    assertEquals(3, fileByLoaded.getBlocks().length);
 +    assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize());
 +
 +    if (isUC) {
 +      assertEquals(client,
 +          fileByLoaded.getFileUnderConstructionFeature().getClientName());
 +      assertEquals(clientMachine,
 +          fileByLoaded.getFileUnderConstructionFeature().getClientMachine());
 +    }
 +  }
 +
 +  /**
 +   * Test if a INodeFile with BlockInfoStriped can be saved by
 +   * FSImageSerialization and loaded by FSImageFormat#Loader.
 +   */
 +  @Test
 +  public void testSaveAndLoadStripedINodeFile() throws IOException{
 +    Configuration conf = new Configuration();
 +    MiniDFSCluster cluster = null;
 +    try {
 +      cluster = new MiniDFSCluster.Builder(conf).build();
 +      cluster.waitActive();
 +      testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, false);
 +    } finally {
 +      if (cluster != null) {
 +        cluster.shutdown();
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Test if a INodeFileUnderConstruction with BlockInfoStriped can be
 +   * saved and loaded by FSImageSerialization
 +   */
 +  @Test
-   public void testSaveAndLoadStripedINodeFileUC() throws IOException{
++  public void testSaveAndLoadStripedINodeFileUC() throws IOException {
 +    // construct a INode with StripedBlock for saving and loading
 +    Configuration conf = new Configuration();
 +    MiniDFSCluster cluster = null;
 +    try {
 +      cluster = new MiniDFSCluster.Builder(conf).build();
 +      cluster.waitActive();
 +      testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, true);
 +    } finally {
 +      if (cluster != null) {
 +        cluster.shutdown();
 +      }
 +    }
 +  }
 +
+    /**
+    * On checkpointing , stale fsimage checkpoint file should be deleted.
+    */
+   @Test
+   public void testRemovalStaleFsimageCkpt() throws IOException {
+     MiniDFSCluster cluster = null;
+     SecondaryNameNode secondary = null;
+     Configuration conf = new HdfsConfiguration();
+     try {
+       cluster = new MiniDFSCluster.Builder(conf).
+           numDataNodes(1).format(true).build();
+       conf.set(DFSConfigKeys.DFS_NAMENODE_SECONDARY_HTTP_ADDRESS_KEY,
+           "0.0.0.0:0");
+       secondary = new SecondaryNameNode(conf);
+       // Do checkpointing
+       secondary.doCheckpoint();
+       NNStorage storage = secondary.getFSImage().storage;
+       File currentDir = FSImageTestUtil.
+           getCurrentDirs(storage, NameNodeDirType.IMAGE).get(0);
+       // Create a stale fsimage.ckpt file
+       File staleCkptFile = new File(currentDir.getPath() +
+           "/fsimage.ckpt_0000000000000000002");
+       staleCkptFile.createNewFile();
+       assertTrue(staleCkptFile.exists());
+       // After checkpoint stale fsimage.ckpt file should be deleted
+       secondary.doCheckpoint();
+       assertFalse(staleCkptFile.exists());
+     } finally {
+       if (secondary != null) {
+         secondary.shutdown();
+         secondary = null;
+       }
+       if (cluster != null) {
+         cluster.shutdown();
+         cluster = null;
+       }
+     }
+   }
+ 
    /**
     * Ensure that the digest written by the saver equals to the digest of the
     * file.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index 53a5d67,767f4de..f6aae22
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@@ -54,8 -54,7 +54,8 @@@ import org.apache.hadoop.hdfs.protocol.
  import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
  import org.apache.hadoop.hdfs.protocol.LocatedBlock;
  import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
  import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
  import org.apache.hadoop.security.UserGroupInformation;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index e6f505e,8818f17..30f7f5c
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@@ -1202,16 -1198,15 +1202,16 @@@ public class TestFsck 
  
      HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication,
          blockSize, modTime, accessTime, perms, owner, group, symlink,
 -        path, fileId, numChildren, null, storagePolicy);
 -    Result res = new Result(conf);
 +        path, fileId, numChildren, null, storagePolicy, null);
 +    Result replRes = new ReplicationResult(conf);
 +    Result ecRes = new ErasureCodingResult(conf);
  
      try {
 -      fsck.check(pathString, file, res);
 +      fsck.check(pathString, file, replRes, ecRes);
      } catch (Exception e) {
-       fail("Unexpected exception "+ e.getMessage());
+       fail("Unexpected exception " + e.getMessage());
      }
 -    assertTrue(res.toString().contains("HEALTHY"));
 +    assertTrue(replRes.isHealthy());
    }
  
    /** Test fsck with symlinks in the filesystem */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
index b1bb191,0000000..de29997
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java
@@@ -1,125 -1,0 +1,125 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hdfs.server.namenode;
 +
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.FSDataOutputStream;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.fs.StorageType;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.DFSTestUtil;
 +import org.apache.hadoop.hdfs.DistributedFileSystem;
 +import org.apache.hadoop.hdfs.MiniDFSCluster;
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 +import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 +import org.apache.hadoop.io.IOUtils;
 +import org.junit.After;
 +import org.junit.Assert;
 +import org.junit.Before;
 +import org.junit.Test;
 +
 +import java.io.IOException;
 +
 +/**
 + * Make sure we correctly update the quota usage with the striped blocks.
 + */
 +public class TestQuotaWithStripedBlocks {
 +  private static final int BLOCK_SIZE = 1024 * 1024;
 +  private static final long DISK_QUOTA = BLOCK_SIZE * 10;
 +  private static final ErasureCodingPolicy ecPolicy =
 +      ErasureCodingPolicyManager.getSystemDefaultPolicy();
 +  private static final int NUM_DATA_BLOCKS = ecPolicy.getNumDataUnits();
 +  private static final int NUM_PARITY_BLOCKS = ecPolicy.getNumParityUnits();
 +  private static final int GROUP_SIZE = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS;
 +  private static final Path ecDir = new Path("/ec");
 +
 +  private MiniDFSCluster cluster;
 +  private FSDirectory dir;
 +  private DistributedFileSystem dfs;
 +
 +  @Before
 +  public void setUp() throws IOException {
 +    final Configuration conf = new Configuration();
 +    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
 +    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE).build();
 +    cluster.waitActive();
 +
 +    dir = cluster.getNamesystem().getFSDirectory();
 +    dfs = cluster.getFileSystem();
 +
 +    dfs.mkdirs(ecDir);
 +    dfs.getClient().createErasureCodingZone(ecDir.toString(), ecPolicy);
 +    dfs.setQuota(ecDir, Long.MAX_VALUE - 1, DISK_QUOTA);
 +    dfs.setQuotaByStorageType(ecDir, StorageType.DISK, DISK_QUOTA);
-     dfs.setStoragePolicy(ecDir, HdfsServerConstants.HOT_STORAGE_POLICY_NAME);
++    dfs.setStoragePolicy(ecDir, HdfsConstants.HOT_STORAGE_POLICY_NAME);
 +  }
 +
 +  @After
 +  public void tearDown() {
 +    if (cluster != null) {
 +      cluster.shutdown();
 +    }
 +  }
 +
 +  @Test
 +  public void testUpdatingQuotaCount() throws Exception {
 +    final Path file = new Path(ecDir, "file");
 +    FSDataOutputStream out = null;
 +
 +    try {
 +      out = dfs.create(file, (short) 1);
 +
 +      INodeFile fileNode = dir.getINode4Write(file.toString()).asFile();
 +      ExtendedBlock previous = null;
 +      // Create striped blocks which have a cell in each block.
 +      Block newBlock = DFSTestUtil.addStripedBlockToFile(cluster.getDataNodes(),
 +          dfs, cluster.getNamesystem(), file.toString(), fileNode,
 +          dfs.getClient().getClientName(), previous, 1);
 +      previous = new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(),
 +          newBlock);
 +
 +      final INodeDirectory dirNode = dir.getINode4Write(ecDir.toString())
 +          .asDirectory();
 +      final long spaceUsed = dirNode.getDirectoryWithQuotaFeature()
 +          .getSpaceConsumed().getStorageSpace();
 +      final long diskUsed = dirNode.getDirectoryWithQuotaFeature()
 +          .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK);
 +      // When we add a new block we update the quota using the full block size.
 +      Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, spaceUsed);
 +      Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, diskUsed);
 +
 +      dfs.getClient().getNamenode().complete(file.toString(),
 +          dfs.getClient().getClientName(), previous, fileNode.getId());
 +
 +      final long actualSpaceUsed = dirNode.getDirectoryWithQuotaFeature()
 +          .getSpaceConsumed().getStorageSpace();
 +      final long actualDiskUsed = dirNode.getDirectoryWithQuotaFeature()
 +          .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK);
 +      // In this case the file's real size is cell size * block group size.
 +      Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
 +          actualSpaceUsed);
 +      Assert.assertEquals(HdfsConstants.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE,
 +          actualDiskUsed);
 +    } finally {
 +      IOUtils.cleanup(null, out);
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
index 7684c1d,0000000..8b1a11f
mode 100644,000000..100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java
@@@ -1,283 -1,0 +1,283 @@@
 +/**
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.hadoop.hdfs.server.namenode;
 +
 +import static org.junit.Assert.assertEquals;
 +import static org.junit.Assert.assertTrue;
 +import static org.junit.Assert.assertFalse;
 +
 +import java.io.IOException;
 +
 +import org.apache.commons.logging.Log;
 +import org.apache.commons.logging.LogFactory;
 +import org.apache.hadoop.conf.Configuration;
 +import org.apache.hadoop.fs.Path;
 +import org.apache.hadoop.fs.permission.FsPermission;
 +import org.apache.hadoop.fs.permission.PermissionStatus;
 +import org.apache.hadoop.hdfs.DFSConfigKeys;
 +import org.apache.hadoop.hdfs.DFSTestUtil;
 +import org.apache.hadoop.hdfs.DistributedFileSystem;
 +import org.apache.hadoop.hdfs.MiniDFSCluster;
 +import org.apache.hadoop.hdfs.protocol.Block;
 +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 +import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionStriped;
++import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 +
 +import org.junit.Test;
 +
 +/**
 + * This class tests INodeFile with striped feature.
 + */
 +public class TestStripedINodeFile {
 +  public static final Log LOG = LogFactory.getLog(TestINodeFile.class);
 +
 +  private static final PermissionStatus perm = new PermissionStatus(
 +      "userName", null, FsPermission.getDefault());
 +
 +  private final BlockStoragePolicySuite defaultSuite =
 +      BlockStoragePolicySuite.createDefaultSuite();
 +  private final BlockStoragePolicy defaultPolicy =
 +      defaultSuite.getDefaultPolicy();
 +
 +  private static final ErasureCodingPolicy testECPolicy
 +      = ErasureCodingPolicyManager.getSystemDefaultPolicy();
 +
 +  private static INodeFile createStripedINodeFile() {
 +    return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L,
 +        null, (short)0, 1024L, HdfsServerConstants.COLD_STORAGE_POLICY_ID, true);
 +  }
 +
 +  @Test
 +  public void testBlockStripedFeature()
 +      throws IOException, InterruptedException{
 +    INodeFile inf = createStripedINodeFile();
 +    assertTrue(inf.isStriped());
 +  }
 +
 +  @Test
 +  public void testBlockStripedTotalBlockCount() {
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    assertEquals(9, blockInfoStriped.getTotalBlockNum());
 +  }
 +
 +  @Test
 +  public void testBlockStripedLength()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    inf.addBlock(blockInfoStriped);
 +    assertEquals(1, inf.getBlocks().length);
 +  }
 +
 +  @Test
 +  public void testBlockStripedConsumedSpace()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(1);
 +    inf.addBlock(blockInfoStriped);
 +    //   0. Calculate the total bytes per stripes <Num Bytes per Stripes>
 +    //   1. Calculate the number of stripes in this block group. <Num Stripes>
 +    //   2. Calculate the last remaining length which does not make a stripe. <Last Stripe Length>
 +    //   3. Total consumed space is the total of
 +    //     a. The total of the full cells of data blocks and parity blocks.
 +    //     b. The remaining of data block which does not make a stripe.
 +    //     c. The last parity block cells. These size should be same
 +    //        to the first cell in this stripe.
 +    // So the total consumed space is the sum of
 +    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
 +    //  b. <Num Bytes> % <Num Bytes per Stripes> = 1
 +    //  c. <Last Stripe Length> * <Parity Block Num> = 1 * 3
 +    assertEquals(4, inf.storagespaceConsumedStriped().getStorageSpace());
 +    assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
 +  }
 +
 +  @Test
 +  public void testMultipleBlockStripedConsumedSpace()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk1 = new Block(1);
 +    BlockInfoStriped blockInfoStriped1
 +        = new BlockInfoStriped(blk1, testECPolicy);
 +    blockInfoStriped1.setNumBytes(1);
 +    Block blk2 = new Block(2);
 +    BlockInfoStriped blockInfoStriped2
 +        = new BlockInfoStriped(blk2, testECPolicy);
 +    blockInfoStriped2.setNumBytes(1);
 +    inf.addBlock(blockInfoStriped1);
 +    inf.addBlock(blockInfoStriped2);
 +    // This is the double size of one block in above case.
 +    assertEquals(4 * 2, inf.storagespaceConsumedStriped().getStorageSpace());
 +    assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace());
 +  }
 +
 +  @Test
 +  public void testBlockStripedFileSize()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(100);
 +    inf.addBlock(blockInfoStriped);
 +    // Compute file size should return actual data
 +    // size which is retained by this file.
 +    assertEquals(100, inf.computeFileSize());
 +    assertEquals(100, inf.computeFileSize(false, false));
 +  }
 +
 +  @Test
-   public void testBlockUCStripedFileSize()
++  public void testBlockStripedUCFileSize()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
-     BlockInfoUnderConstructionStriped bInfoUCStriped
-         = new BlockInfoUnderConstructionStriped(blk, testECPolicy);
++    BlockInfoStripedUnderConstruction bInfoUCStriped
++        = new BlockInfoStripedUnderConstruction(blk, testECPolicy);
 +    bInfoUCStriped.setNumBytes(100);
 +    inf.addBlock(bInfoUCStriped);
 +    assertEquals(100, inf.computeFileSize());
 +    assertEquals(0, inf.computeFileSize(false, false));
 +  }
 +
 +  @Test
 +  public void testBlockStripedComputeQuotaUsage()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
 +    BlockInfoStriped blockInfoStriped
 +        = new BlockInfoStriped(blk, testECPolicy);
 +    blockInfoStriped.setNumBytes(100);
 +    inf.addBlock(blockInfoStriped);
 +
 +    QuotaCounts counts =
 +        inf.computeQuotaUsageWithStriped(defaultPolicy,
 +            new QuotaCounts.Builder().build());
 +    assertEquals(1, counts.getNameSpace());
 +    // The total consumed space is the sum of
 +    //  a. <Cell Size> * (<Num Stripes> - 1) * <Total Block Num> = 0
 +    //  b. <Num Bytes> % <Num Bytes per Stripes> = 100
 +    //  c. <Last Stripe Length> * <Parity Block Num> = 100 * 3
 +    assertEquals(400, counts.getStorageSpace());
 +  }
 +
 +  @Test
-   public void testBlockUCStripedComputeQuotaUsage()
++  public void testBlockStripedUCComputeQuotaUsage()
 +      throws IOException, InterruptedException {
 +    INodeFile inf = createStripedINodeFile();
 +    Block blk = new Block(1);
-     BlockInfoUnderConstructionStriped bInfoUCStriped
-         = new BlockInfoUnderConstructionStriped(blk, testECPolicy);
++    BlockInfoStripedUnderConstruction bInfoUCStriped
++        = new BlockInfoStripedUnderConstruction(blk, testECPolicy);
 +    bInfoUCStriped.setNumBytes(100);
 +    inf.addBlock(bInfoUCStriped);
 +
 +    QuotaCounts counts
 +        = inf.computeQuotaUsageWithStriped(defaultPolicy,
 +              new QuotaCounts.Builder().build());
 +    assertEquals(1024, inf.getPreferredBlockSize());
 +    assertEquals(1, counts.getNameSpace());
-     // Consumed space in the case of BlockInfoUCStriped can be calculated
++    // Consumed space in the case of BlockInfoStripedUC can be calculated
 +    // by using preferred block size. This is 1024 and total block num
 +    // is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216.
 +    assertEquals(9216, counts.getStorageSpace());
 +  }
 +
 +  /**
 +   * Test the behavior of striped and contiguous block deletions.
 +   */
 +  @Test(timeout = 60000)
 +  public void testDeleteOp() throws Exception {
 +    MiniDFSCluster cluster = null;
 +    try {
 +      final int len = 1024;
 +      final Path parentDir = new Path("/parentDir");
 +      final Path zone = new Path(parentDir, "zone");
 +      final Path zoneFile = new Path(zone, "zoneFile");
 +      final Path contiguousFile = new Path(parentDir, "someFile");
 +      final DistributedFileSystem dfs;
 +      final Configuration conf = new Configuration();
 +      final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS
 +          + HdfsConstants.NUM_PARITY_BLOCKS;
 +      conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, 2);
 +
 +      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE)
 +          .build();
 +      cluster.waitActive();
 +
 +      FSNamesystem fsn = cluster.getNamesystem();
 +      dfs = cluster.getFileSystem();
 +      dfs.mkdirs(zone);
 +
 +      // create erasure zone
 +      dfs.createErasureCodingZone(zone, null);
 +      DFSTestUtil.createFile(dfs, zoneFile, len, (short) 1, 0xFEED);
 +      DFSTestUtil.createFile(dfs, contiguousFile, len, (short) 1, 0xFEED);
 +      final FSDirectory fsd = fsn.getFSDirectory();
 +
 +      // Case-1: Verify the behavior of striped blocks
 +      // Get blocks of striped file
 +      INode inodeStriped = fsd.getINode("/parentDir/zone/zoneFile");
 +      assertTrue("Failed to get INodeFile for /parentDir/zone/zoneFile",
 +          inodeStriped instanceof INodeFile);
 +      INodeFile inodeStripedFile = (INodeFile) inodeStriped;
 +      BlockInfo[] stripedBlks = inodeStripedFile.getBlocks();
 +      for (BlockInfo blockInfo : stripedBlks) {
 +        assertFalse("Mistakenly marked the block as deleted!",
 +            blockInfo.isDeleted());
 +      }
 +
 +      // delete erasure zone directory
 +      dfs.delete(zone, true);
 +      for (BlockInfo blockInfo : stripedBlks) {
 +        assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted());
 +      }
 +
 +      // Case-2: Verify the behavior of contiguous blocks
 +      // Get blocks of contiguous file
 +      INode inode = fsd.getINode("/parentDir/someFile");
 +      assertTrue("Failed to get INodeFile for /parentDir/someFile",
 +          inode instanceof INodeFile);
 +      INodeFile inodeFile = (INodeFile) inode;
 +      BlockInfo[] contiguousBlks = inodeFile.getBlocks();
 +      for (BlockInfo blockInfo : contiguousBlks) {
 +        assertFalse("Mistakenly marked the block as deleted!",
 +            blockInfo.isDeleted());
 +      }
 +
 +      // delete parent directory
 +      dfs.delete(parentDir, true);
 +      for (BlockInfo blockInfo : contiguousBlks) {
 +        assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted());
 +      }
 +    } finally {
 +      if (cluster != null) {
 +        cluster.shutdown();
 +      }
 +    }
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6b6a63bb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
----------------------------------------------------------------------
diff --cc hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
index 38d0daa,a1abd08..128bd9e
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
@@@ -43,8 -43,8 +43,9 @@@ import org.apache.hadoop.hdfs.DFSTestUt
  import org.apache.hadoop.hdfs.DistributedFileSystem;
  import org.apache.hadoop.hdfs.MiniDFSCluster;
  import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
- import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
+ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
++import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
  import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
  import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
  import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
@@@ -175,9 -175,9 +176,9 @@@ public class SnapshotTestHelper 
     * localName (className@hashCode) parent permission group user
     * 
     * Specific information for different types of INode: 
--   * {@link INodeDirectory}:childrenSize 
-    * {@link INodeFile}: fileSize, block list. Check {@link BlockInfoContiguous#toString()}
-    * and {@link BlockInfoUnderConstructionContiguous#toString()} for detailed information.
++   * {@link INodeDirectory}:childrenSize
+    * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()}
+    * and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information.
     * {@link FileWithSnapshot}: next link
     * </pre>
     * @see INode#dumpTreeRecursively()