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 zj...@apache.org on 2015/06/18 20:24:39 UTC

[01/50] [abbrv] hadoop git commit: HDFS-8499. Refactor BlockInfo class hierarchy with static helper class. Contributed by Zhe Zhang.

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2928 9137aeae0 -> 8c036a14e


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --git 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
index 6b8388e..d081a6b 100644
--- 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,7 +22,7 @@ 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.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 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;
@@ -133,7 +133,7 @@ public class FileDiffList extends
     Block dontRemoveBlock = null;
     if (lastBlock != null && lastBlock.getBlockUCState().equals(
         HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {
-      dontRemoveBlock = ((BlockInfoContiguousUnderConstruction) lastBlock)
+      dontRemoveBlock = ((BlockInfoUnderConstruction) lastBlock)
           .getTruncateBlock();
     }
     // Collect the remaining blocks of the file, ignoring truncate block

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 50b85c0..d06b024 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -109,7 +109,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -1612,9 +1612,9 @@ public class DFSTestUtil {
     BlockInfo storedBlock = bm0.getStoredBlock(blk.getLocalBlock());
     assertTrue("Block " + blk + " should be under construction, " +
         "got: " + storedBlock,
-        storedBlock instanceof BlockInfoContiguousUnderConstruction);
-    BlockInfoContiguousUnderConstruction ucBlock =
-      (BlockInfoContiguousUnderConstruction)storedBlock;
+        storedBlock instanceof BlockInfoUnderConstruction);
+    BlockInfoUnderConstruction ucBlock =
+      (BlockInfoUnderConstruction)storedBlock;
     // We expect that the replica with the most recent heart beat will be
     // the one to be in charge of the synchronization / recovery protocol.
     final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
index a7ba293..630cd1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
@@ -23,7 +23,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 /**
@@ -40,7 +39,8 @@ public class TestBlockInfoUnderConstruction {
     DatanodeDescriptor dd3 = s3.getDatanodeDescriptor();
 
     dd1.isAlive = dd2.isAlive = dd3.isAlive = true;
-    BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
+    BlockInfoUnderConstruction blockInfo =
+        new BlockInfoUnderConstructionContiguous(
         new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP),
         (short) 3,
         BlockUCState.UNDER_CONSTRUCTION,
@@ -51,7 +51,7 @@ public class TestBlockInfoUnderConstruction {
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, -2 * 1000);
     blockInfo.initializeBlockRecovery(1);
-    BlockInfoContiguousUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
+    BlockInfoUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
     // Recovery attempt #2.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index fd03745..5a82b15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -726,7 +726,7 @@ public class TestBlockManager {
     // verify the storage info is correct
     assertTrue(bm.getStoredBlock(new Block(receivedBlockId)).findStorageInfo
         (ds) >= 0);
-    assertTrue(((BlockInfoContiguousUnderConstruction) bm.
+    assertTrue(((BlockInfoUnderConstruction) bm.
         getStoredBlock(new Block(receivingBlockId))).getNumExpectedLocations() > 0);
     assertTrue(bm.getStoredBlock(new Block(receivingReceivedBlockId))
         .findStorageInfo(ds) >= 0);
@@ -747,8 +747,8 @@ public class TestBlockManager {
 
   private BlockInfo addUcBlockToBM(long blkId) {
     Block block = new Block(blkId);
-    BlockInfoContiguousUnderConstruction blockInfo =
-        new BlockInfoContiguousUnderConstruction(block, (short) 3);
+    BlockInfoUnderConstruction blockInfo =
+        new BlockInfoUnderConstructionContiguous(block, (short) 3);
     BlockCollection bc = Mockito.mock(BlockCollection.class);
     Mockito.doReturn((short) 3).when(bc).getPreferredBlockReplication();
     bm.blocksMap.addBlockCollection(blockInfo, bc);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
index 6fc30ba..e48e9e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java
@@ -39,7 +39,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
-import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 /**
@@ -173,7 +172,8 @@ public class TestHeartbeatHandling {
               dd1.getStorageInfos()[0],
               dd2.getStorageInfos()[0],
               dd3.getStorageInfos()[0]};
-          BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
+          BlockInfoUnderConstruction blockInfo =
+              new BlockInfoUnderConstructionContiguous(
               new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
               BlockUCState.UNDER_RECOVERY, storages);
           dd1.addBlockToBeRecovered(blockInfo);
@@ -195,7 +195,7 @@ public class TestHeartbeatHandling {
           // More than the default stale interval of 30 seconds.
           DFSTestUtil.resetLastUpdatesWithOffset(dd2, -40 * 1000);
           DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
-          blockInfo = new BlockInfoContiguousUnderConstruction(
+          blockInfo = new BlockInfoUnderConstructionContiguous(
               new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
               BlockUCState.UNDER_RECOVERY, storages);
           dd1.addBlockToBeRecovered(blockInfo);
@@ -216,7 +216,7 @@ public class TestHeartbeatHandling {
           // More than the default stale interval of 30 seconds.
           DFSTestUtil.resetLastUpdatesWithOffset(dd2, - 40 * 1000);
           DFSTestUtil.resetLastUpdatesWithOffset(dd3, - 80 * 1000);
-          blockInfo = new BlockInfoContiguousUnderConstruction(
+          blockInfo = new BlockInfoUnderConstructionContiguous(
               new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
               BlockUCState.UNDER_RECOVERY, storages);
           dd1.addBlockToBeRecovered(blockInfo);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git 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
index fccaf3c..6e98538 100644
--- 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
@@ -1176,7 +1176,8 @@ public class TestReplicationPolicy {
     // block under construction, the BlockManager will realize the expected
     // replication has been achieved and remove it from the under-replicated
     // queue.
-    BlockInfoContiguousUnderConstruction info = new BlockInfoContiguousUnderConstruction(block1, (short) 1);
+    BlockInfoUnderConstruction info =
+        new BlockInfoUnderConstructionContiguous(block1, (short) 1);
     BlockCollection bc = mock(BlockCollection.class);
     when(bc.getPreferredBlockReplication()).thenReturn((short)1);
     bm.addBlockCollection(info, bc);
@@ -1232,7 +1233,7 @@ public class TestReplicationPolicy {
 
     DatanodeStorageInfo[] storageAry = {new DatanodeStorageInfo(
         dataNodes[0], new DatanodeStorage("s1"))};
-    final BlockInfoContiguousUnderConstruction ucBlock =
+    final BlockInfoUnderConstruction ucBlock =
         info.convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION,
             storageAry);
     DatanodeStorageInfo storage = mock(DatanodeStorageInfo.class);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
index f372bec..872ff9c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
 import org.junit.AfterClass;
@@ -170,7 +170,7 @@ public class TestBlockUnderConstruction {
       final List<LocatedBlock> blocks = lb.getLocatedBlocks();
       assertEquals(i, blocks.size());
       final Block b = blocks.get(blocks.size() - 1).getBlock().getLocalBlock();
-      assertTrue(b instanceof BlockInfoContiguousUnderConstruction);
+      assertTrue(b instanceof BlockInfoUnderConstruction);
 
       if (++i < NUM_BLOCKS) {
         // write one more block

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
index bea7241..b6cb522 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
@@ -24,7 +24,8 @@ import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.junit.Test;
@@ -68,8 +69,10 @@ public class TestCommitBlockSynchronization {
     namesystem.dir.getINodeMap().put(file);
 
     FSNamesystem namesystemSpy = spy(namesystem);
-    BlockInfoContiguousUnderConstruction blockInfo = new BlockInfoContiguousUnderConstruction(
-        block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets);
+    BlockInfoUnderConstruction blockInfo =
+        new BlockInfoUnderConstructionContiguous(
+        block, (short) 1, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+            targets);
     blockInfo.setBlockCollection(file);
     blockInfo.setGenerationStamp(genStamp);
     blockInfo.initializeBlockRecovery(genStamp);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git 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
index 222f22b..df920e0 100644
--- 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,7 +54,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 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.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -1019,7 +1019,7 @@ public class TestFileTruncate {
           is(fsn.getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
-      long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
+      long blockRecoveryId = ((BlockInfoUnderConstruction) file.getLastBlock())
           .getBlockRecoveryId();
       assertThat(blockRecoveryId, is(initialGenStamp + 1));
       fsn.getEditLog().logTruncate(
@@ -1052,7 +1052,7 @@ public class TestFileTruncate {
           is(fsn.getBlockIdManager().getGenerationStampV2()));
       assertThat(file.getLastBlock().getBlockUCState(),
           is(HdfsServerConstants.BlockUCState.UNDER_RECOVERY));
-      long blockRecoveryId = ((BlockInfoContiguousUnderConstruction) file.getLastBlock())
+      long blockRecoveryId = ((BlockInfoUnderConstruction) file.getLastBlock())
           .getBlockRecoveryId();
       assertThat(blockRecoveryId, is(initialGenStamp + 1));
       fsn.getEditLog().logTruncate(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
index d202fb7..14d9a1e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java
@@ -72,7 +72,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
@@ -752,8 +752,8 @@ public class TestRetryCacheWithHA {
     boolean checkNamenodeBeforeReturn() throws Exception {
       INodeFile fileNode = cluster.getNamesystem(0).getFSDirectory()
           .getINode4Write(file).asFile();
-      BlockInfoContiguousUnderConstruction blkUC =
-          (BlockInfoContiguousUnderConstruction) (fileNode.getBlocks())[1];
+      BlockInfoUnderConstruction blkUC =
+          (BlockInfoUnderConstruction) (fileNode.getBlocks())[1];
       int datanodeNum = blkUC.getExpectedStorageLocations().length;
       for (int i = 0; i < CHECKTIMES && datanodeNum != 2; i++) {
         Thread.sleep(1000);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java
----------------------------------------------------------------------
diff --git 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
index a1abd08..824f45b 100644
--- 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
@@ -44,7 +44,7 @@ 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.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
 import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
@@ -177,7 +177,7 @@ public class SnapshotTestHelper {
    * Specific information for different types of INode: 
    * {@link INodeDirectory}:childrenSize 
    * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()}
-   * and {@link BlockInfoContiguousUnderConstruction#toString()} for detailed information.
+   * and {@link BlockInfoUnderConstruction#toString()} for detailed information.
    * {@link FileWithSnapshot}: next link
    * </pre>
    * @see INode#dumpTreeRecursively()


[21/50] [abbrv] hadoop git commit: MAPREDUCE-6363. [NNBench] Lease mismatch error when running with multiple mappers. Contributed by Brahma Reddy Battula.

Posted by zj...@apache.org.
MAPREDUCE-6363. [NNBench] Lease mismatch error when running with multiple mappers. Contributed by Brahma Reddy Battula.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9fd568bd
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9fd568bd
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9fd568bd

Branch: refs/heads/YARN-2928
Commit: 9fd568bd0b118df95759d2514b375f111ce745fc
Parents: 77bbe95
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Jun 15 14:52:26 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:06 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                 |  3 +++
 .../test/java/org/apache/hadoop/hdfs/NNBench.java    | 15 +++++++++++----
 2 files changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fd568bd/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index ead80a0..3c2e8f9 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -495,6 +495,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6389. Fix BaileyBorweinPlouffe CLI usage message. (Brahma Reddy Battula
     via devaraj)
 
+    MAPREDUCE-6363. [NNBench] Lease mismatch error when running with multiple
+    mappers. (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fd568bd/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
index bb50213..6c8ec17 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/hdfs/NNBench.java
@@ -57,6 +57,9 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.mapred.OutputCollector;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Reducer;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TaskAttemptID;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 
 /**
  * This program executes a specified operation that applies load to 
@@ -687,6 +690,9 @@ public class NNBench {
       dataDirName = conf.get("test.nnbench.datadir.name");
       op = conf.get("test.nnbench.operation");
       readFile = conf.getBoolean("test.nnbench.readFileAfterOpen", false);
+      int taskId =
+          TaskAttemptID.forName(conf.get(MRJobConfig.TASK_ATTEMPT_ID))
+              .getTaskID().getId();
       
       long totalTimeTPmS = 0l;
       long startTimeTPmS = 0l;
@@ -699,18 +705,19 @@ public class NNBench {
       successfulFileOps = 0l;
       
       if (barrier()) {
+        String filePrefix = "file_" + taskId + "_";
         if (op.equals(OP_CREATE_WRITE)) {
           startTimeTPmS = System.currentTimeMillis();
-          doCreateWriteOp("file_" + hostName + "_", reporter);
+          doCreateWriteOp(filePrefix, reporter);
         } else if (op.equals(OP_OPEN_READ)) {
           startTimeTPmS = System.currentTimeMillis();
-          doOpenReadOp("file_" + hostName + "_", reporter);
+          doOpenReadOp(filePrefix, reporter);
         } else if (op.equals(OP_RENAME)) {
           startTimeTPmS = System.currentTimeMillis();
-          doRenameOp("file_" + hostName + "_", reporter);
+          doRenameOp(filePrefix, reporter);
         } else if (op.equals(OP_DELETE)) {
           startTimeTPmS = System.currentTimeMillis();
-          doDeleteOp("file_" + hostName + "_", reporter);
+          doDeleteOp(filePrefix, reporter);
         }
         
         endTimeTPms = System.currentTimeMillis();


[04/50] [abbrv] hadoop git commit: HDFS-7923. The DataNodes should rate-limit their full block reports by asking the NN on heartbeat messages (cmccabe)

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java
----------------------------------------------------------------------
diff --git 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
index 92c329e..ff70c3f 100644
--- 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
@@ -109,7 +109,7 @@ public class TestDeadDatanode {
         BlockListAsLongs.EMPTY) };
     try {
       dnp.blockReport(reg, poolId, report,
-          new BlockReportContext(1, 0, System.nanoTime()));
+          new BlockReportContext(1, 0, System.nanoTime(), 0L));
       fail("Expected IOException is not thrown");
     } catch (IOException ex) {
       // Expected
@@ -120,8 +120,8 @@ public class TestDeadDatanode {
     StorageReport[] rep = { new StorageReport(
         new DatanodeStorage(reg.getDatanodeUuid()),
         false, 0, 0, 0, 0) };
-    DatanodeCommand[] cmd = dnp.sendHeartbeat(reg, rep, 0L, 0L, 0, 0, 0, null)
-        .getCommands();
+    DatanodeCommand[] cmd =
+        dnp.sendHeartbeat(reg, rep, 0L, 0L, 0, 0, 0, null, true).getCommands();
     assertEquals(1, cmd.length);
     assertEquals(cmd[0].getAction(), RegisterCommand.REGISTER
         .getAction());


[11/50] [abbrv] hadoop git commit: HDFS-8593. Calculation of effective layout version mishandles comparison to current layout version in storage. Contributed by Chris Nauroth.

Posted by zj...@apache.org.
HDFS-8593. Calculation of effective layout version mishandles comparison to current layout version in storage. Contributed by Chris Nauroth.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fb7f8ec1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fb7f8ec1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fb7f8ec1

Branch: refs/heads/YARN-2928
Commit: fb7f8ec15eab00fe1548d6d6cd758124fc0ea83c
Parents: d6ff0e8
Author: cnauroth <cn...@apache.org>
Authored: Sat Jun 13 10:37:52 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:05 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../hdfs/server/namenode/FSNamesystem.java      | 17 ++++++++++++-----
 .../hdfs/server/namenode/TestFSNamesystem.java  | 20 ++++++++++++++++++++
 3 files changed, 35 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb7f8ec1/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 51a0897..9aabd34 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -886,6 +886,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8554. TestDatanodeLayoutUpgrade fails on Windows. (cnauroth)
 
+    HDFS-8593. Calculation of effective layout version mishandles comparison to
+    current layout version in storage. (cnauroth)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb7f8ec1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index ef53692..f962373 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -7004,10 +7004,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @return layout version in effect
    */
   public int getEffectiveLayoutVersion() {
-    if (isRollingUpgrade()) {
-      int storageLV = fsImage.getStorage().getLayoutVersion();
-      if (storageLV >=
-          NameNodeLayoutVersion.MINIMUM_COMPATIBLE_LAYOUT_VERSION) {
+    return getEffectiveLayoutVersion(isRollingUpgrade(),
+        fsImage.getStorage().getLayoutVersion(),
+        NameNodeLayoutVersion.MINIMUM_COMPATIBLE_LAYOUT_VERSION,
+        NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
+  }
+
+  @VisibleForTesting
+  static int getEffectiveLayoutVersion(boolean isRollingUpgrade, int storageLV,
+      int minCompatLV, int currentLV) {
+    if (isRollingUpgrade) {
+      if (storageLV <= minCompatLV) {
         // The prior layout version satisfies the minimum compatible layout
         // version of the current software.  Keep reporting the prior layout
         // as the effective one.  Downgrade is possible.
@@ -7016,7 +7023,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
     // The current software cannot satisfy the layout version of the prior
     // software.  Proceed with using the current layout version.
-    return NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
+    return currentLV;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fb7f8ec1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
index 8b0662c..26bb4f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java
@@ -213,4 +213,24 @@ public class TestFSNamesystem {
     fsn.imageLoadComplete();
     assertTrue(fsn.isImageLoaded());
   }
+
+  @Test
+  public void testGetEffectiveLayoutVersion() {
+    assertEquals(-63,
+        FSNamesystem.getEffectiveLayoutVersion(true, -60, -61, -63));
+    assertEquals(-61,
+        FSNamesystem.getEffectiveLayoutVersion(true, -61, -61, -63));
+    assertEquals(-62,
+        FSNamesystem.getEffectiveLayoutVersion(true, -62, -61, -63));
+    assertEquals(-63,
+        FSNamesystem.getEffectiveLayoutVersion(true, -63, -61, -63));
+    assertEquals(-63,
+        FSNamesystem.getEffectiveLayoutVersion(false, -60, -61, -63));
+    assertEquals(-63,
+        FSNamesystem.getEffectiveLayoutVersion(false, -61, -61, -63));
+    assertEquals(-63,
+        FSNamesystem.getEffectiveLayoutVersion(false, -62, -61, -63));
+    assertEquals(-63,
+        FSNamesystem.getEffectiveLayoutVersion(false, -63, -61, -63));
+  }
 }


[24/50] [abbrv] hadoop git commit: YARN-3711. Documentation of ResourceManager HA should explain configurations about listen addresses. Contributed by Masatake Iwasaki.

Posted by zj...@apache.org.
YARN-3711. Documentation of ResourceManager HA should explain configurations about listen addresses. Contributed by Masatake Iwasaki.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5032eb97
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5032eb97
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5032eb97

Branch: refs/heads/YARN-2928
Commit: 5032eb979bc3c7b3d8b9ab9388f47bc6676fca8f
Parents: 5e24417
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Jun 16 10:12:38 2015 +0900
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:07 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                     |  3 +++
 .../src/site/markdown/ResourceManagerHA.md          | 16 +++++++++++++++-
 2 files changed, 18 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5032eb97/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1776a89..99f2c64 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -644,6 +644,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3723. Need to clearly document primaryFilter and otherInfo value type.
     (Zhijie Shen via xgong)
 
+    YARN-3711. Documentation of ResourceManager HA should explain configurations
+    about listen addresses. (Masatake Iwasaki via ozawa)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5032eb97/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
index 596cba7..49669a1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/ResourceManagerHA.md
@@ -67,7 +67,13 @@ Most of the failover functionality is tunable using various configuration proper
 | `yarn.resourcemanager.zk-address` | Address of the ZK-quorum. Used both for the state-store and embedded leader-election. |
 | `yarn.resourcemanager.ha.enabled` | Enable RM HA. |
 | `yarn.resourcemanager.ha.rm-ids` | List of logical IDs for the RMs. e.g., "rm1,rm2". |
-| `yarn.resourcemanager.hostname.*rm-id*` | For each *rm-id*, specify the hostname the RM corresponds to. Alternately, one could set each of the RM's service addresses. |
+| `yarn.resourcemanager.hostname.`*rm-id* | For each *rm-id*, specify the hostname the RM corresponds to. Alternately, one could set each of the RM's service addresses. |
+| `yarn.resourcemanager.address.`*rm-id* | For each *rm-id*, specify host:port for clients to submit jobs. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
+| `yarn.resourcemanager.scheduler.address.`*rm-id* | For each *rm-id*, specify scheduler host:port for ApplicationMasters to obtain resources. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
+| `yarn.resourcemanager.resource-tracker.address.`*rm-id* | For each *rm-id*, specify host:port for NodeManagers to connect. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
+| `yarn.resourcemanager.admin.address.`*rm-id* | For each *rm-id*, specify host:port for administrative commands. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
+| `yarn.resourcemanager.webapp.address.`*rm-id* | For each *rm-id*, specify host:port of the RM web application corresponds to. You do not need this if you set `yarn.http.policy` to `HTTPS_ONLY`. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
+| `yarn.resourcemanager.webapp.https.address.`*rm-id* | For each *rm-id*, specify host:port of the RM https web application corresponds to. You do not need this if you set `yarn.http.policy` to `HTTP_ONLY`. If set, overrides the hostname set in `yarn.resourcemanager.hostname.`*rm-id*. |
 | `yarn.resourcemanager.ha.id` | Identifies the RM in the ensemble. This is optional; however, if set, admins have to ensure that all the RMs have their own IDs in the config. |
 | `yarn.resourcemanager.ha.automatic-failover.enabled` | Enable automatic failover; By default, it is enabled only when HA is enabled. |
 | `yarn.resourcemanager.ha.automatic-failover.embedded` | Use embedded leader-elector to pick the Active RM, when automatic failover is enabled. By default, it is enabled only when HA is enabled. |
@@ -105,6 +111,14 @@ Here is the sample of minimal setup for RM failover.
   <value>master2</value>
 </property>
 <property>
+  <name>yarn.resourcemanager.webapp.address.rm1</name>
+  <value>master1:8088</value>
+</property>
+<property>
+  <name>yarn.resourcemanager.webapp.address.rm2</name>
+  <value>master2:8088</value>
+</property>
+<property>
   <name>yarn.resourcemanager.zk-address</name>
   <value>zk1:2181,zk2:2181,zk3:2181</value>
 </property>


[13/50] [abbrv] hadoop git commit: HDFS-8607. TestFileCorruption doesn't work as expected. (Contributed by Walter Su)

Posted by zj...@apache.org.
HDFS-8607. TestFileCorruption doesn't work as expected. (Contributed by Walter Su)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e447ae3f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e447ae3f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e447ae3f

Branch: refs/heads/YARN-2928
Commit: e447ae3f2923a7c4cb82672553b3cf6384601bac
Parents: 30d1fb0
Author: Arpit Agarwal <ar...@apache.org>
Authored: Mon Jun 15 10:11:53 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:05 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt      |  3 +++
 .../apache/hadoop/hdfs/TestFileCorruption.java   | 19 +++++++++++--------
 2 files changed, 14 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e447ae3f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index cb2679a..c98d918 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -889,6 +889,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8593. Calculation of effective layout version mishandles comparison to
     current layout version in storage. (cnauroth)
 
+    HDFS-8607. TestFileCorruption doesn't work as expected. (Walter Su via
+    Arpit Agarwal)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e447ae3f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
index 8001bfb..8e0ffe7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCorruption.java
@@ -27,8 +27,12 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileOutputStream;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.DirectoryFileFilter;
+import org.apache.commons.io.filefilter.PrefixFileFilter;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.fs.FileSystem;
@@ -74,14 +78,13 @@ public class TestFileCorruption {
       String bpid = cluster.getNamesystem().getBlockPoolId();
       File data_dir = MiniDFSCluster.getFinalizedDir(storageDir, bpid);
       assertTrue("data directory does not exist", data_dir.exists());
-      File[] blocks = data_dir.listFiles();
-      assertTrue("Blocks do not exist in data-dir", (blocks != null) && (blocks.length > 0));
-      for (int idx = 0; idx < blocks.length; idx++) {
-        if (!blocks[idx].getName().startsWith(Block.BLOCK_FILE_PREFIX)) {
-          continue;
-        }
-        System.out.println("Deliberately removing file "+blocks[idx].getName());
-        assertTrue("Cannot remove file.", blocks[idx].delete());
+      Collection<File> blocks = FileUtils.listFiles(data_dir,
+          new PrefixFileFilter(Block.BLOCK_FILE_PREFIX),
+          DirectoryFileFilter.DIRECTORY);
+      assertTrue("Blocks do not exist in data-dir", blocks.size() > 0);
+      for (File block : blocks) {
+        System.out.println("Deliberately removing file " + block.getName());
+        assertTrue("Cannot remove file.", block.delete());
       }
       assertTrue("Corrupted replicas not handled properly.",
                  util.checkFiles(fs, "/srcdat"));


[49/50] [abbrv] hadoop git commit: MAPREDUCE-6373. The logger reports total input paths but it is referring to input files. Contributed by Bibin A Chundatt.

Posted by zj...@apache.org.
MAPREDUCE-6373. The logger reports total input paths but it is referring
to input files. Contributed by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/05d63e67
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/05d63e67
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/05d63e67

Branch: refs/heads/YARN-2928
Commit: 05d63e6727508baa0a747838f5d41fd45f5b4e06
Parents: 8119664
Author: Devaraj K <de...@apache.org>
Authored: Thu Jun 18 11:42:22 2015 +0530
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:02 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../src/main/java/org/apache/hadoop/mapred/FileInputFormat.java   | 2 +-
 .../org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java    | 3 +--
 3 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d63e67/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 5b66604..f885542 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -501,6 +501,9 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-6363. [NNBench] Lease mismatch error when running with multiple
     mappers. (Brahma Reddy Battula via aajisaka)
 
+    MAPREDUCE-6373. The logger reports total input paths but it is referring
+    to input files. (Bibin A Chundatt via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d63e67/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
index c6cbd50..2c58ebe 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java
@@ -253,7 +253,7 @@ public abstract class FileInputFormat<K, V> implements InputFormat<K, V> {
       LOG.debug("Time taken to get FileStatuses: "
           + sw.now(TimeUnit.MILLISECONDS));
     }
-    LOG.info("Total input paths to process : " + result.length);
+    LOG.info("Total input files to process : " + result.length);
     return result;
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/05d63e67/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
index f5cd5d1..0c5ede9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.mapred.LocatedFileStatusFetcher;
-import org.apache.hadoop.mapred.SplitLocationInfo;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.Job;
@@ -287,7 +286,7 @@ public abstract class FileInputFormat<K, V> extends InputFormat<K, V> {
       LOG.debug("Time taken to get FileStatuses: "
           + sw.now(TimeUnit.MILLISECONDS));
     }
-    LOG.info("Total input paths to process : " + result.size()); 
+    LOG.info("Total input files to process : " + result.size());
     return result;
   }
 


[18/50] [abbrv] hadoop git commit: HDFS-8361. Choose SSD over DISK in block placement.

Posted by zj...@apache.org.
HDFS-8361. Choose SSD over DISK in block placement.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5e24417a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5e24417a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5e24417a

Branch: refs/heads/YARN-2928
Commit: 5e24417a5c01654e818e59940a73aa960c3d5f0d
Parents: b181b87
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 15 17:12:01 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:06 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/fs/StorageType.java  |  7 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 +
 .../hadoop/hdfs/TestBlockStoragePolicy.java     | 75 +++++++++++++++++++-
 3 files changed, 80 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e24417a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
index 68069d7..0948801 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
@@ -33,10 +33,11 @@ import org.apache.hadoop.util.StringUtils;
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 public enum StorageType {
-  DISK(false),
+  // sorted by the speed of the storage types, from fast to slow
+  RAM_DISK(true),
   SSD(false),
-  ARCHIVE(false),
-  RAM_DISK(true);
+  DISK(false),
+  ARCHIVE(false);
 
   private final boolean isTransient;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e24417a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 9822575..79e7820 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -920,6 +920,8 @@ Release 2.7.1 - UNRELEASED
     HDFS-8521. Add VisibleForTesting annotation to
     BlockPoolSlice#selectReplicaToDelete. (cmccabe)
 
+    HDFS-8361. Choose SSD over DISK in block placement.  (szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e24417a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index ea69f97..0d59ded 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -26,6 +26,7 @@ import java.util.*;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.FSDataOutputStream;
@@ -40,7 +41,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper;
-import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.net.Node;
@@ -1153,6 +1154,50 @@ public class TestBlockStoragePolicy {
   }
 
   @Test
+  public void testChooseSsdOverDisk() throws Exception {
+    BlockStoragePolicy policy = new BlockStoragePolicy((byte) 9, "TEST1",
+        new StorageType[]{StorageType.SSD, StorageType.DISK,
+            StorageType.ARCHIVE}, new StorageType[]{}, new StorageType[]{});
+
+    final String[] racks = {"/d1/r1", "/d1/r1", "/d1/r1"};
+    final String[] hosts = {"host1", "host2", "host3"};
+    final StorageType[] disks = {StorageType.DISK, StorageType.DISK, StorageType.DISK};
+
+    final DatanodeStorageInfo[] diskStorages
+        = DFSTestUtil.createDatanodeStorageInfos(3, racks, hosts, disks);
+    final DatanodeDescriptor[] dataNodes
+        = DFSTestUtil.toDatanodeDescriptor(diskStorages);
+    for(int i = 0; i < dataNodes.length; i++) {
+      BlockManagerTestUtil.updateStorage(dataNodes[i],
+          new DatanodeStorage("ssd" + i, DatanodeStorage.State.NORMAL,
+              StorageType.SSD));
+    }
+
+    FileSystem.setDefaultUri(conf, "hdfs://localhost:0");
+    conf.set(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY, "0.0.0.0:0");
+    File baseDir = PathUtils.getTestDir(TestReplicationPolicy.class);
+    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
+        new File(baseDir, "name").getPath());
+    DFSTestUtil.formatNameNode(conf);
+    NameNode namenode = new NameNode(conf);
+
+    final BlockManager bm = namenode.getNamesystem().getBlockManager();
+    BlockPlacementPolicy replicator = bm.getBlockPlacementPolicy();
+    NetworkTopology cluster = bm.getDatanodeManager().getNetworkTopology();
+    for (DatanodeDescriptor datanode : dataNodes) {
+      cluster.add(datanode);
+    }
+
+    DatanodeStorageInfo[] targets = replicator.chooseTarget("/foo", 3,
+        dataNodes[0], Collections.<DatanodeStorageInfo>emptyList(), false,
+        new HashSet<Node>(), 0, policy);
+    System.out.println(policy.getName() + ": " + Arrays.asList(targets));
+    Assert.assertEquals(2, targets.length);
+    Assert.assertEquals(StorageType.SSD, targets[0].getStorageType());
+    Assert.assertEquals(StorageType.DISK, targets[1].getStorageType());
+  }
+
+  @Test
   public void testGetFileStoragePolicyAfterRestartNN() throws Exception {
     //HDFS8219
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
@@ -1233,4 +1278,32 @@ public class TestBlockStoragePolicy {
       cluster.shutdown();
     }
   }
+
+  @Test
+  public void testStorageType() {
+    final EnumMap<StorageType, Integer> map = new EnumMap<>(StorageType.class);
+
+    //put storage type is reversed order
+    map.put(StorageType.ARCHIVE, 1);
+    map.put(StorageType.DISK, 1);
+    map.put(StorageType.SSD, 1);
+    map.put(StorageType.RAM_DISK, 1);
+
+    {
+      final Iterator<StorageType> i = map.keySet().iterator();
+      Assert.assertEquals(StorageType.RAM_DISK, i.next());
+      Assert.assertEquals(StorageType.SSD, i.next());
+      Assert.assertEquals(StorageType.DISK, i.next());
+      Assert.assertEquals(StorageType.ARCHIVE, i.next());
+    }
+
+    {
+      final Iterator<Map.Entry<StorageType, Integer>> i
+          = map.entrySet().iterator();
+      Assert.assertEquals(StorageType.RAM_DISK, i.next().getKey());
+      Assert.assertEquals(StorageType.SSD, i.next().getKey());
+      Assert.assertEquals(StorageType.DISK, i.next().getKey());
+      Assert.assertEquals(StorageType.ARCHIVE, i.next().getKey());
+    }
+  }
 }


[14/50] [abbrv] hadoop git commit: HDFS-8595. TestCommitBlockSynchronization fails in branch-2.7. (Patch applies to all branches). (Contributed by Arpit Agarwal)

Posted by zj...@apache.org.
HDFS-8595. TestCommitBlockSynchronization fails in branch-2.7. (Patch applies to all branches). (Contributed by Arpit Agarwal)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/30d1fb0c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/30d1fb0c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/30d1fb0c

Branch: refs/heads/YARN-2928
Commit: 30d1fb0ca7e375de14dce3b69695d87713cc4316
Parents: 2b93ab3
Author: Arpit Agarwal <ar...@apache.org>
Authored: Sun Jun 14 16:24:30 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:05 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hdfs/server/namenode/TestCommitBlockSynchronization.java      | 1 +
 2 files changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/30d1fb0c/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 1caa8c5..cb2679a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1005,6 +1005,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8596. TestDistributedFileSystem et al tests are broken in branch-2
     due to incorrect setting of "datanode" attribute. (Yongjun Zhang)
 
+    HDFS-8595. TestCommitBlockSynchronization fails in branch-2.7. (Patch
+    applies to all branches). (Arpit Agarwal)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/30d1fb0c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
index b6cb522..9de426e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockSynchronization.java
@@ -78,6 +78,7 @@ public class TestCommitBlockSynchronization {
     blockInfo.initializeBlockRecovery(genStamp);
     doReturn(blockInfo).when(file).removeLastBlock(any(Block.class));
     doReturn(true).when(file).isUnderConstruction();
+    doReturn(new BlockInfoContiguous[1]).when(file).getBlocks();
 
     doReturn(blockInfo).when(namesystemSpy).getStoredBlock(any(Block.class));
     doReturn(blockInfo).when(file).getLastBlock();


[29/50] [abbrv] hadoop git commit: HADOOP-12095. org.apache.hadoop.fs.shell.TestCount fails. Contributed by Brahma Reddy Battula.

Posted by zj...@apache.org.
HADOOP-12095. org.apache.hadoop.fs.shell.TestCount fails. Contributed by Brahma Reddy Battula.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5e962f68
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5e962f68
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5e962f68

Branch: refs/heads/YARN-2928
Commit: 5e962f682228937995fa6539d3f086e317073378
Parents: 12b46e1
Author: Akira Ajisaka <aa...@apache.org>
Authored: Tue Jun 16 13:38:03 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:18:58 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                  | 3 +++
 .../src/test/java/org/apache/hadoop/fs/shell/TestCount.java      | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e962f68/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 11c2f2a..9a16a7d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -858,6 +858,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12001. Fixed LdapGroupsMapping to include configurable Posix UID and
     GID attributes during the search. (Patrick White via vinodkv)
 
+    HADOOP-12095. org.apache.hadoop.fs.shell.TestCount fails.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5e962f68/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
index 44fc1e6..116c071 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
@@ -281,7 +281,7 @@ public class TestCount {
     count.processOptions(options);
     String withStorageTypeHeader =
         // <----13---> <-------17------> <----13-----> <------17------->
-        "   DISK_QUOTA    REM_DISK_QUOTA     SSD_QUOTA     REM_SSD_QUOTA " +
+        "    SSD_QUOTA     REM_SSD_QUOTA    DISK_QUOTA    REM_DISK_QUOTA " +
         // <----13---> <-------17------>
         "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA " +
         "PATHNAME";
@@ -335,8 +335,8 @@ public class TestCount {
     count.processOptions(options);
     String withStorageTypeHeader =
         // <----13---> <-------17------>
-        "   DISK_QUOTA    REM_DISK_QUOTA " +
         "    SSD_QUOTA     REM_SSD_QUOTA " +
+        "   DISK_QUOTA    REM_DISK_QUOTA " +
         "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA " +
         "PATHNAME";
     verify(out).println(withStorageTypeHeader);


[17/50] [abbrv] hadoop git commit: Move HDFS-8540 to 2.8 in CHANGES.txt.

Posted by zj...@apache.org.
Move HDFS-8540 to 2.8 in CHANGES.txt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b181b870
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b181b870
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b181b870

Branch: refs/heads/YARN-2928
Commit: b181b870409cb93d6b7c12c2be311bbbb84f2e8c
Parents: fe8250d
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 15 17:01:55 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:06 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b181b870/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 584d94d..9822575 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -629,6 +629,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8499. Refactor BlockInfo class hierarchy with static helper class.
     (Zhe Zhang via wang)
 
+    HDFS-8540.  Mover should exit with NO_MOVE_BLOCK if no block can be moved.
+    (surendra singh lilhore via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
@@ -917,9 +920,6 @@ Release 2.7.1 - UNRELEASED
     HDFS-8521. Add VisibleForTesting annotation to
     BlockPoolSlice#selectReplicaToDelete. (cmccabe)
 
-    HDFS-8540.  Mover should exit with NO_MOVE_BLOCK if no block can be moved.
-    (surendra singh lilhore via szetszwo)
-
   OPTIMIZATIONS
 
   BUG FIXES


[05/50] [abbrv] hadoop git commit: HDFS-7923. The DataNodes should rate-limit their full block reports by asking the NN on heartbeat messages (cmccabe)

Posted by zj...@apache.org.
HDFS-7923. The DataNodes should rate-limit their full block reports by asking the NN on heartbeat messages (cmccabe)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f7460011
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f7460011
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f7460011

Branch: refs/heads/YARN-2928
Commit: f7460011855bda4ec76f6761aa369ca52ec497a0
Parents: 9eb54e4
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Jun 12 11:17:51 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:04 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   4 +
 .../DatanodeProtocolClientSideTranslatorPB.java |   8 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   3 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |   3 +-
 .../server/blockmanagement/BlockManager.java    |  41 ++-
 .../BlockManagerFaultInjector.java              |  52 +++
 .../BlockReportLeaseManager.java                | 355 +++++++++++++++++++
 .../server/blockmanagement/DatanodeManager.java |   2 +
 .../hdfs/server/datanode/BPServiceActor.java    |  71 +++-
 .../hadoop/hdfs/server/datanode/DNConf.java     |   4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  11 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |   9 +-
 .../server/protocol/BlockReportContext.java     |  25 +-
 .../hdfs/server/protocol/DatanodeProtocol.java  |   5 +-
 .../hdfs/server/protocol/HeartbeatResponse.java |  10 +-
 .../hdfs/server/protocol/RegisterCommand.java   |   2 +-
 .../src/main/proto/DatanodeProtocol.proto       |   6 +
 .../src/main/resources/hdfs-default.xml         |  21 ++
 .../hdfs/protocol/TestBlockListAsLongs.java     |   4 +-
 .../TestBlockReportRateLimiting.java            | 246 +++++++++++++
 .../blockmanagement/TestDatanodeManager.java    |  21 +-
 .../TestNameNodePrunesMissingStorages.java      |   2 +-
 .../server/datanode/TestBPOfferService.java     |   7 +-
 .../TestBlockHasMultipleReplicasOnSameDN.java   |   2 +-
 .../hdfs/server/datanode/TestBlockRecovery.java |   6 +-
 .../datanode/TestBpServiceActorScheduler.java   |   2 +-
 .../TestDatanodeProtocolRetryPolicy.java        |   8 +-
 .../server/datanode/TestFsDatasetCache.java     |   9 +-
 .../TestNNHandlesBlockReportPerStorage.java     |   2 +-
 .../TestNNHandlesCombinedBlockReport.java       |   2 +-
 .../hdfs/server/datanode/TestStorageReport.java |   2 +-
 .../server/namenode/NNThroughputBenchmark.java  |   8 +-
 .../hdfs/server/namenode/NameNodeAdapter.java   |   2 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |   6 +-
 34 files changed, 890 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 5bb6e53..3f72608 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -434,6 +434,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT = 0;
   public static final String  DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY = "dfs.blockreport.split.threshold";
   public static final long    DFS_BLOCKREPORT_SPLIT_THRESHOLD_DEFAULT = 1000 * 1000;
+  public static final String  DFS_NAMENODE_MAX_FULL_BLOCK_REPORT_LEASES = "dfs.namenode.max.full.block.report.leases";
+  public static final int     DFS_NAMENODE_MAX_FULL_BLOCK_REPORT_LEASES_DEFAULT = 6;
+  public static final String  DFS_NAMENODE_FULL_BLOCK_REPORT_LEASE_LENGTH_MS = "dfs.namenode.full.block.report.lease.length.ms";
+  public static final long    DFS_NAMENODE_FULL_BLOCK_REPORT_LEASE_LENGTH_MS_DEFAULT = 5L * 60L * 1000L;
   public static final String  DFS_CACHEREPORT_INTERVAL_MSEC_KEY = "dfs.cachereport.intervalMsec";
   public static final long    DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT = 10 * 1000;
   public static final String  DFS_BLOCK_INVALIDATE_LIMIT_KEY = "dfs.block.invalidate.limit";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index 825e835..94028a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -132,11 +132,13 @@ public class DatanodeProtocolClientSideTranslatorPB implements
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration registration,
       StorageReport[] reports, long cacheCapacity, long cacheUsed,
       int xmitsInProgress, int xceiverCount, int failedVolumes,
-      VolumeFailureSummary volumeFailureSummary) throws IOException {
+      VolumeFailureSummary volumeFailureSummary,
+      boolean requestFullBlockReportLease) throws IOException {
     HeartbeatRequestProto.Builder builder = HeartbeatRequestProto.newBuilder()
         .setRegistration(PBHelper.convert(registration))
         .setXmitsInProgress(xmitsInProgress).setXceiverCount(xceiverCount)
-        .setFailedVolumes(failedVolumes);
+        .setFailedVolumes(failedVolumes)
+        .setRequestFullBlockReportLease(requestFullBlockReportLease);
     builder.addAllReports(PBHelper.convertStorageReports(reports));
     if (cacheCapacity != 0) {
       builder.setCacheCapacity(cacheCapacity);
@@ -165,7 +167,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       rollingUpdateStatus = PBHelper.convert(resp.getRollingUpgradeStatus());
     }
     return new HeartbeatResponse(cmds, PBHelper.convert(resp.getHaStatus()),
-        rollingUpdateStatus);
+        rollingUpdateStatus, resp.getFullBlockReportLeaseId());
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 873eb6d..e133ec7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -114,7 +114,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
           report, request.getCacheCapacity(), request.getCacheUsed(),
           request.getXmitsInProgress(),
           request.getXceiverCount(), request.getFailedVolumes(),
-          volumeFailureSummary);
+          volumeFailureSummary, request.getRequestFullBlockReportLease());
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -135,6 +135,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       builder.setRollingUpgradeStatus(PBHelper
           .convertRollingUpgradeStatus(rollingUpdateStatus));
     }
+    builder.setFullBlockReportLeaseId(response.getFullBlockReportLeaseId());
     return builder.build();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index c9a9c33..32d9614 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -3042,7 +3042,7 @@ public class PBHelper {
 
   public static BlockReportContext convert(BlockReportContextProto proto) {
     return new BlockReportContext(proto.getTotalRpcs(),
-        proto.getCurRpc(), proto.getId());
+        proto.getCurRpc(), proto.getId(), proto.getLeaseId());
   }
 
   public static BlockReportContextProto convert(BlockReportContext context) {
@@ -3050,6 +3050,7 @@ public class PBHelper {
         setTotalRpcs(context.getTotalRpcs()).
         setCurRpc(context.getCurRpc()).
         setId(context.getReportId()).
+        setLeaseId(context.getLeaseId()).
         build();
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index d18d7fe..4562d94 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -75,6 +75,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State;
 import org.apache.hadoop.hdfs.server.protocol.KeyUpdateCommand;
@@ -124,6 +125,7 @@ public class BlockManager {
   private final AtomicLong excessBlocksCount = new AtomicLong(0L);
   private final AtomicLong postponedMisreplicatedBlocksCount = new AtomicLong(0L);
   private final long startupDelayBlockDeletionInMs;
+  private final BlockReportLeaseManager blockReportLeaseManager;
 
   /** Used by metrics */
   public long getPendingReplicationBlocksCount() {
@@ -348,7 +350,8 @@ public class BlockManager {
     this.numBlocksPerIteration = conf.getInt(
         DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT,
         DFSConfigKeys.DFS_BLOCK_MISREPLICATION_PROCESSING_LIMIT_DEFAULT);
-    
+    this.blockReportLeaseManager = new BlockReportLeaseManager(conf);
+
     LOG.info("defaultReplication         = " + defaultReplication);
     LOG.info("maxReplication             = " + maxReplication);
     LOG.info("minReplication             = " + minReplication);
@@ -1712,7 +1715,28 @@ public class BlockManager {
        */
     }
   }
-  
+
+  public long requestBlockReportLeaseId(DatanodeRegistration nodeReg) {
+    assert namesystem.hasReadLock();
+    DatanodeDescriptor node = null;
+    try {
+      node = datanodeManager.getDatanode(nodeReg);
+    } catch (UnregisteredNodeException e) {
+      LOG.warn("Unregistered datanode {}", nodeReg);
+      return 0;
+    }
+    if (node == null) {
+      LOG.warn("Failed to find datanode {}", nodeReg);
+      return 0;
+    }
+    // Request a new block report lease.  The BlockReportLeaseManager has
+    // its own internal locking.
+    long leaseId = blockReportLeaseManager.requestLease(node);
+    BlockManagerFaultInjector.getInstance().
+        requestBlockReportLease(node, leaseId);
+    return leaseId;
+  }
+
   /**
    * StatefulBlockInfo is used to build the "toUC" list, which is a list of
    * updates to the information about under-construction blocks.
@@ -1817,6 +1841,12 @@ public class BlockManager {
             + " because namenode still in startup phase", nodeID);
         return !node.hasStaleStorages();
       }
+      if (context != null) {
+        if (!blockReportLeaseManager.checkLease(node, startTime,
+              context.getLeaseId())) {
+          return false;
+        }
+      }
 
       if (storageInfo.getBlockReportCount() == 0) {
         // The first block report can be processed a lot more efficiently than
@@ -1835,6 +1865,9 @@ public class BlockManager {
         if (lastStorageInRpc) {
           int rpcsSeen = node.updateBlockReportContext(context);
           if (rpcsSeen >= context.getTotalRpcs()) {
+            long leaseId = blockReportLeaseManager.removeLease(node);
+            BlockManagerFaultInjector.getInstance().
+                removeBlockReportLease(node, leaseId);
             List<DatanodeStorageInfo> zombies = node.removeZombieStorages();
             if (zombies.isEmpty()) {
               LOG.debug("processReport 0x{}: no zombie storages found.",
@@ -3845,4 +3878,8 @@ public class BlockManager {
     clearQueues();
     blocksMap.clear();
   }
+
+  public BlockReportLeaseManager getBlockReportLeaseManager() {
+    return blockReportLeaseManager;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerFaultInjector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerFaultInjector.java
new file mode 100644
index 0000000..957c5c0
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerFaultInjector.java
@@ -0,0 +1,52 @@
+/**
+ * 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.blockmanagement;
+
+import java.io.IOException;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+
+/**
+ * Used to inject certain faults for testing.
+ */
+public class BlockManagerFaultInjector {
+  @VisibleForTesting
+  public static BlockManagerFaultInjector instance =
+      new BlockManagerFaultInjector();
+
+  @VisibleForTesting
+  public static BlockManagerFaultInjector getInstance() {
+    return instance;
+  }
+
+  @VisibleForTesting
+  public void incomingBlockReportRpc(DatanodeID nodeID,
+          BlockReportContext context) throws IOException {
+
+  }
+
+  @VisibleForTesting
+  public void requestBlockReportLease(DatanodeDescriptor node, long leaseId) {
+  }
+
+  @VisibleForTesting
+  public void removeBlockReportLease(DatanodeDescriptor node, long leaseId) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java
new file mode 100644
index 0000000..cd037f5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java
@@ -0,0 +1,355 @@
+/**
+ * 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.blockmanagement;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.util.Time;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashMap;
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * The BlockReportLeaseManager manages block report leases.<p/>
+ *
+ * DataNodes request BR leases from the NameNode by sending a heartbeat with
+ * the requestBlockReportLease field set.  The NameNode may choose to respond
+ * with a non-zero lease ID.  If so, that DataNode can send a block report with
+ * the given lease ID for the next few minutes.  The NameNode will accept
+ * these full block reports.<p/>
+ *
+ * BR leases limit the number of incoming full block reports to the NameNode
+ * at any given time.  For compatibility reasons, the NN will always accept
+ * block reports sent with a lease ID of 0 and queue them for processing
+ * immediately.  Full block reports which were manually triggered will also
+ * have a lease ID of 0, bypassing the rate-limiting.<p/>
+ *
+ * Block report leases expire after a certain amount of time.  This mechanism
+ * is in place so that a DN which dies while holding a lease does not
+ * permanently decrease the number of concurrent block reports which the NN is
+ * willing to accept.<p/>
+ *
+ * When considering which DNs to grant a BR lease, the NameNode gives priority
+ * to the DNs which have gone the longest without sending a full block
+ * report.<p/>
+ */
+class BlockReportLeaseManager {
+  static final Logger LOG =
+      LoggerFactory.getLogger(BlockReportLeaseManager.class);
+
+  private static class NodeData {
+    /**
+     * The UUID of the datanode.
+     */
+    final String datanodeUuid;
+
+    /**
+     * The lease ID, or 0 if there is no lease.
+     */
+    long leaseId;
+
+    /**
+     * The time when the lease was issued, or 0 if there is no lease.
+     */
+    long leaseTimeMs;
+
+    /**
+     * Previous element in the list.
+     */
+    NodeData prev;
+
+    /**
+     * Next element in the list.
+     */
+    NodeData next;
+
+    static NodeData ListHead(String name) {
+      NodeData node = new NodeData(name);
+      node.next = node;
+      node.prev = node;
+      return node;
+    }
+
+    NodeData(String datanodeUuid) {
+      this.datanodeUuid = datanodeUuid;
+    }
+
+    void removeSelf() {
+      if (this.prev != null) {
+        this.prev.next = this.next;
+      }
+      if (this.next != null) {
+        this.next.prev = this.prev;
+      }
+      this.next = null;
+      this.prev = null;
+    }
+
+    void addToEnd(NodeData node) {
+      Preconditions.checkState(node.next == null);
+      Preconditions.checkState(node.prev == null);
+      node.prev = this.prev;
+      node.next = this;
+      this.prev.next = node;
+      this.prev = node;
+    }
+
+    void addToBeginning(NodeData node) {
+      Preconditions.checkState(node.next == null);
+      Preconditions.checkState(node.prev == null);
+      node.next = this.next;
+      node.prev = this;
+      this.next.prev = node;
+      this.next = node;
+    }
+  }
+
+  /**
+   * List of datanodes which don't currently have block report leases.
+   */
+  private final NodeData deferredHead = NodeData.ListHead("deferredHead");
+
+  /**
+   * List of datanodes which currently have block report leases.
+   */
+  private final NodeData pendingHead = NodeData.ListHead("pendingHead");
+
+  /**
+   * Maps datanode UUIDs to NodeData.
+   */
+  private final HashMap<String, NodeData> nodes = new HashMap<>();
+
+  /**
+   * The current length of the pending list.
+   */
+  private int numPending = 0;
+
+  /**
+   * The maximum number of leases to hand out at any given time.
+   */
+  private final int maxPending;
+
+  /**
+   * The number of milliseconds after which a lease will expire.
+   */
+  private final long leaseExpiryMs;
+
+  /**
+   * The next ID we will use for a block report lease.
+   */
+  private long nextId = ThreadLocalRandom.current().nextLong();
+
+  BlockReportLeaseManager(Configuration conf) {
+    this(conf.getInt(
+          DFSConfigKeys.DFS_NAMENODE_MAX_FULL_BLOCK_REPORT_LEASES,
+          DFSConfigKeys.DFS_NAMENODE_MAX_FULL_BLOCK_REPORT_LEASES_DEFAULT),
+        conf.getLong(
+          DFSConfigKeys.DFS_NAMENODE_FULL_BLOCK_REPORT_LEASE_LENGTH_MS,
+          DFSConfigKeys.DFS_NAMENODE_FULL_BLOCK_REPORT_LEASE_LENGTH_MS_DEFAULT));
+  }
+
+  BlockReportLeaseManager(int maxPending, long leaseExpiryMs) {
+    Preconditions.checkArgument(maxPending >= 1,
+        "Cannot set the maximum number of block report leases to a " +
+            "value less than 1.");
+    this.maxPending = maxPending;
+    Preconditions.checkArgument(leaseExpiryMs >= 1,
+        "Cannot set full block report lease expiry period to a value " +
+         "less than 1.");
+    this.leaseExpiryMs = leaseExpiryMs;
+  }
+
+  /**
+   * Get the next block report lease ID.  Any number is valid except 0.
+   */
+  private synchronized long getNextId() {
+    long id;
+    do {
+      id = nextId++;
+    } while (id == 0);
+    return id;
+  }
+
+  public synchronized void register(DatanodeDescriptor dn) {
+    registerNode(dn);
+  }
+
+  private synchronized NodeData registerNode(DatanodeDescriptor dn) {
+    if (nodes.containsKey(dn.getDatanodeUuid())) {
+      LOG.info("Can't register DN {} because it is already registered.",
+          dn.getDatanodeUuid());
+      return null;
+    }
+    NodeData node = new NodeData(dn.getDatanodeUuid());
+    deferredHead.addToBeginning(node);
+    nodes.put(dn.getDatanodeUuid(), node);
+    LOG.info("Registered DN {} ({}).", dn.getDatanodeUuid(), dn.getXferAddr());
+    return node;
+  }
+
+  private synchronized void remove(NodeData node) {
+    if (node.leaseId != 0) {
+      numPending--;
+      node.leaseId = 0;
+      node.leaseTimeMs = 0;
+    }
+    node.removeSelf();
+  }
+
+  public synchronized void unregister(DatanodeDescriptor dn) {
+    NodeData node = nodes.remove(dn.getDatanodeUuid());
+    if (node == null) {
+      LOG.info("Can't unregister DN {} because it is not currently " +
+          "registered.", dn.getDatanodeUuid());
+      return;
+    }
+    remove(node);
+  }
+
+  public synchronized long requestLease(DatanodeDescriptor dn) {
+    NodeData node = nodes.get(dn.getDatanodeUuid());
+    if (node == null) {
+      LOG.warn("DN {} ({}) requested a lease even though it wasn't yet " +
+          "registered.  Registering now.", dn.getDatanodeUuid(),
+          dn.getXferAddr());
+      node = registerNode(dn);
+    }
+    if (node.leaseId != 0) {
+      // The DataNode wants a new lease, even though it already has one.
+      // This can happen if the DataNode is restarted in between requesting
+      // a lease and using it.
+      LOG.debug("Removing existing BR lease 0x{} for DN {} in order to " +
+               "issue a new one.", Long.toHexString(node.leaseId),
+               dn.getDatanodeUuid());
+    }
+    remove(node);
+    long monotonicNowMs = Time.monotonicNow();
+    pruneExpiredPending(monotonicNowMs);
+    if (numPending >= maxPending) {
+      if (LOG.isDebugEnabled()) {
+        StringBuilder allLeases = new StringBuilder();
+        String prefix = "";
+        for (NodeData cur = pendingHead.next; cur != pendingHead;
+             cur = cur.next) {
+          allLeases.append(prefix).append(cur.datanodeUuid);
+          prefix = ", ";
+        }
+        LOG.debug("Can't create a new BR lease for DN {}, because " +
+              "numPending equals maxPending at {}.  Current leases: {}",
+              dn.getDatanodeUuid(), numPending, allLeases.toString());
+      }
+      return 0;
+    }
+    numPending++;
+    node.leaseId = getNextId();
+    node.leaseTimeMs = monotonicNowMs;
+    pendingHead.addToEnd(node);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Created a new BR lease 0x{} for DN {}.  numPending = {}",
+          Long.toHexString(node.leaseId), dn.getDatanodeUuid(), numPending);
+    }
+    return node.leaseId;
+  }
+
+  private synchronized boolean pruneIfExpired(long monotonicNowMs,
+                                              NodeData node) {
+    if (monotonicNowMs < node.leaseTimeMs + leaseExpiryMs) {
+      return false;
+    }
+    LOG.info("Removing expired block report lease 0x{} for DN {}.",
+        Long.toHexString(node.leaseId), node.datanodeUuid);
+    Preconditions.checkState(node.leaseId != 0);
+    remove(node);
+    deferredHead.addToBeginning(node);
+    return true;
+  }
+
+  private synchronized void pruneExpiredPending(long monotonicNowMs) {
+    NodeData cur = pendingHead.next;
+    while (cur != pendingHead) {
+      NodeData next = cur.next;
+      if (!pruneIfExpired(monotonicNowMs, cur)) {
+        return;
+      }
+      cur = next;
+    }
+    LOG.trace("No entries remaining in the pending list.");
+  }
+
+  public synchronized boolean checkLease(DatanodeDescriptor dn,
+                                         long monotonicNowMs, long id) {
+    if (id == 0) {
+      LOG.debug("Datanode {} is using BR lease id 0x0 to bypass " +
+          "rate-limiting.", dn.getDatanodeUuid());
+      return true;
+    }
+    NodeData node = nodes.get(dn.getDatanodeUuid());
+    if (node == null) {
+      LOG.info("BR lease 0x{} is not valid for unknown datanode {}",
+          Long.toHexString(id), dn.getDatanodeUuid());
+      return false;
+    }
+    if (node.leaseId == 0) {
+      LOG.warn("BR lease 0x{} is not valid for DN {}, because the DN " +
+               "is not in the pending set.",
+               Long.toHexString(id), dn.getDatanodeUuid());
+      return false;
+    }
+    if (pruneIfExpired(monotonicNowMs, node)) {
+      LOG.warn("BR lease 0x{} is not valid for DN {}, because the lease " +
+               "has expired.", Long.toHexString(id), dn.getDatanodeUuid());
+      return false;
+    }
+    if (id != node.leaseId) {
+      LOG.warn("BR lease 0x{} is not valid for DN {}.  Expected BR lease 0x{}.",
+          Long.toHexString(id), dn.getDatanodeUuid(),
+          Long.toHexString(node.leaseId));
+      return false;
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("BR lease 0x{} is valid for DN {}.",
+          Long.toHexString(id), dn.getDatanodeUuid());
+    }
+    return true;
+  }
+
+  public synchronized long removeLease(DatanodeDescriptor dn) {
+    NodeData node = nodes.get(dn.getDatanodeUuid());
+    if (node == null) {
+      LOG.info("Can't remove lease for unknown datanode {}",
+               dn.getDatanodeUuid());
+      return 0;
+    }
+    long id = node.leaseId;
+    if (id == 0) {
+      LOG.debug("DN {} has no lease to remove.", dn.getDatanodeUuid());
+      return 0;
+    }
+    remove(node);
+    deferredHead.addToEnd(node);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Removed BR lease 0x{} for DN {}.  numPending = {}",
+                Long.toHexString(id), dn.getDatanodeUuid(), numPending);
+    }
+    return id;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 01f7972..58349cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -540,6 +540,7 @@ public class DatanodeManager {
     blockManager.removeBlocksAssociatedTo(nodeInfo);
     networktopology.remove(nodeInfo);
     decrementVersionCount(nodeInfo.getSoftwareVersion());
+    blockManager.getBlockReportLeaseManager().unregister(nodeInfo);
 
     if (LOG.isDebugEnabled()) {
       LOG.debug("remove datanode " + nodeInfo);
@@ -602,6 +603,7 @@ public class DatanodeManager {
     networktopology.add(node); // may throw InvalidTopologyException
     host2DatanodeMap.add(node);
     checkIfClusterIsNowMultiRack(node);
+    blockManager.getBlockReportLeaseManager().register(node);
 
     if (LOG.isDebugEnabled()) {
       LOG.debug(getClass().getSimpleName() + ".addDatanode: "

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index 63a0bb6..ea1abbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -29,6 +29,8 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.base.Joiner;
 import org.apache.commons.logging.Log;
@@ -355,9 +357,10 @@ class BPServiceActor implements Runnable {
   void triggerBlockReportForTests() {
     synchronized (pendingIncrementalBRperStorage) {
       scheduler.scheduleHeartbeat();
-      long nextBlockReportTime = scheduler.scheduleBlockReport(0);
+      long oldBlockReportTime = scheduler.nextBlockReportTime;
+      scheduler.forceFullBlockReportNow();
       pendingIncrementalBRperStorage.notifyAll();
-      while (nextBlockReportTime - scheduler.nextBlockReportTime >= 0) {
+      while (oldBlockReportTime == scheduler.nextBlockReportTime) {
         try {
           pendingIncrementalBRperStorage.wait(100);
         } catch (InterruptedException e) {
@@ -419,12 +422,7 @@ class BPServiceActor implements Runnable {
    * @return DatanodeCommands returned by the NN. May be null.
    * @throws IOException
    */
-  List<DatanodeCommand> blockReport() throws IOException {
-    // send block report if timer has expired.
-    if (!scheduler.isBlockReportDue()) {
-      return null;
-    }
-
+  List<DatanodeCommand> blockReport(long fullBrLeaseId) throws IOException {
     final ArrayList<DatanodeCommand> cmds = new ArrayList<DatanodeCommand>();
 
     // Flush any block information that precedes the block report. Otherwise
@@ -460,7 +458,7 @@ class BPServiceActor implements Runnable {
         // Below split threshold, send all reports in a single message.
         DatanodeCommand cmd = bpNamenode.blockReport(
             bpRegistration, bpos.getBlockPoolId(), reports,
-              new BlockReportContext(1, 0, reportId));
+              new BlockReportContext(1, 0, reportId, fullBrLeaseId));
         numRPCs = 1;
         numReportsSent = reports.length;
         if (cmd != null) {
@@ -472,7 +470,8 @@ class BPServiceActor implements Runnable {
           StorageBlockReport singleReport[] = { reports[r] };
           DatanodeCommand cmd = bpNamenode.blockReport(
               bpRegistration, bpos.getBlockPoolId(), singleReport,
-              new BlockReportContext(reports.length, r, reportId));
+              new BlockReportContext(reports.length, r, reportId,
+                  fullBrLeaseId));
           numReportsSent++;
           numRPCs++;
           if (cmd != null) {
@@ -538,7 +537,8 @@ class BPServiceActor implements Runnable {
     return cmd;
   }
   
-  HeartbeatResponse sendHeartBeat() throws IOException {
+  HeartbeatResponse sendHeartBeat(boolean requestBlockReportLease)
+      throws IOException {
     StorageReport[] reports =
         dn.getFSDataset().getStorageReports(bpos.getBlockPoolId());
     if (LOG.isDebugEnabled()) {
@@ -557,7 +557,8 @@ class BPServiceActor implements Runnable {
         dn.getXmitsInProgress(),
         dn.getXceiverCount(),
         numFailedVolumes,
-        volumeFailureSummary);
+        volumeFailureSummary,
+        requestBlockReportLease);
   }
   
   //This must be called only by BPOfferService
@@ -625,8 +626,9 @@ class BPServiceActor implements Runnable {
     LOG.info("For namenode " + nnAddr + " using"
         + " BLOCKREPORT_INTERVAL of " + dnConf.blockReportInterval + "msec"
         + " CACHEREPORT_INTERVAL of " + dnConf.cacheReportInterval + "msec"
-        + " Initial delay: " + dnConf.initialBlockReportDelay + "msec"
+        + " Initial delay: " + dnConf.initialBlockReportDelayMs + "msec"
         + "; heartBeatInterval=" + dnConf.heartBeatInterval);
+    long fullBlockReportLeaseId = 0;
 
     //
     // Now loop for a long time....
@@ -639,6 +641,7 @@ class BPServiceActor implements Runnable {
         // Every so often, send heartbeat or block-report
         //
         final boolean sendHeartbeat = scheduler.isHeartbeatDue(startTime);
+        HeartbeatResponse resp = null;
         if (sendHeartbeat) {
           //
           // All heartbeat messages include following info:
@@ -647,10 +650,23 @@ class BPServiceActor implements Runnable {
           // -- Total capacity
           // -- Bytes remaining
           //
+          boolean requestBlockReportLease = (fullBlockReportLeaseId == 0) &&
+                  scheduler.isBlockReportDue(startTime);
           scheduler.scheduleNextHeartbeat();
           if (!dn.areHeartbeatsDisabledForTests()) {
-            HeartbeatResponse resp = sendHeartBeat();
+            resp = sendHeartBeat(requestBlockReportLease);
             assert resp != null;
+            if (resp.getFullBlockReportLeaseId() != 0) {
+              if (fullBlockReportLeaseId != 0) {
+                LOG.warn(nnAddr + " sent back a full block report lease " +
+                        "ID of 0x" +
+                        Long.toHexString(resp.getFullBlockReportLeaseId()) +
+                        ", but we already have a lease ID of 0x" +
+                        Long.toHexString(fullBlockReportLeaseId) + ". " +
+                        "Overwriting old lease ID.");
+              }
+              fullBlockReportLeaseId = resp.getFullBlockReportLeaseId();
+            }
             dn.getMetrics().addHeartbeat(scheduler.monotonicNow() - startTime);
 
             // If the state of this NN has changed (eg STANDBY->ACTIVE)
@@ -682,7 +698,16 @@ class BPServiceActor implements Runnable {
           reportReceivedDeletedBlocks();
         }
 
-        List<DatanodeCommand> cmds = blockReport();
+        List<DatanodeCommand> cmds = null;
+        boolean forceFullBr =
+            scheduler.forceFullBlockReport.getAndSet(false);
+        if (forceFullBr) {
+          LOG.info("Forcing a full block report to " + nnAddr);
+        }
+        if ((fullBlockReportLeaseId != 0) || forceFullBr) {
+          cmds = blockReport(fullBlockReportLeaseId);
+          fullBlockReportLeaseId = 0;
+        }
         processCommand(cmds == null ? null : cmds.toArray(new DatanodeCommand[cmds.size()]));
 
         DatanodeCommand cmd = cacheReport();
@@ -765,7 +790,7 @@ class BPServiceActor implements Runnable {
     bpos.registrationSucceeded(this, bpRegistration);
 
     // random short delay - helps scatter the BR from all DNs
-    scheduler.scheduleBlockReport(dnConf.initialBlockReportDelay);
+    scheduler.scheduleBlockReport(dnConf.initialBlockReportDelayMs);
   }
 
 
@@ -958,7 +983,7 @@ class BPServiceActor implements Runnable {
     } else {
       LOG.info(bpos.toString() + ": scheduling a full block report.");
       synchronized(pendingIncrementalBRperStorage) {
-        scheduler.scheduleBlockReport(0);
+        scheduler.forceFullBlockReportNow();
         pendingIncrementalBRperStorage.notifyAll();
       }
     }
@@ -1011,6 +1036,9 @@ class BPServiceActor implements Runnable {
     @VisibleForTesting
     boolean resetBlockReportTime = true;
 
+    private final AtomicBoolean forceFullBlockReport =
+        new AtomicBoolean(false);
+
     private final long heartbeatIntervalMs;
     private final long blockReportIntervalMs;
 
@@ -1042,8 +1070,13 @@ class BPServiceActor implements Runnable {
       return (nextHeartbeatTime - startTime <= 0);
     }
 
-    boolean isBlockReportDue() {
-      return nextBlockReportTime - monotonicNow() <= 0;
+    boolean isBlockReportDue(long curTime) {
+      return nextBlockReportTime - curTime <= 0;
+    }
+
+    void forceFullBlockReportNow() {
+      forceFullBlockReport.set(true);
+      resetBlockReportTime = true;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
index 4b7fbc3..42b1b46 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java
@@ -82,7 +82,7 @@ public class DNConf {
   final long heartBeatInterval;
   final long blockReportInterval;
   final long blockReportSplitThreshold;
-  final long initialBlockReportDelay;
+  final long initialBlockReportDelayMs;
   final long cacheReportInterval;
   final long dfsclientSlowIoWarningThresholdMs;
   final long datanodeSlowIoWarningThresholdMs;
@@ -159,7 +159,7 @@ public class DNConf {
           + "greater than or equal to" + "dfs.blockreport.intervalMsec."
           + " Setting initial delay to 0 msec:");
     }
-    initialBlockReportDelay = initBRDelay;
+    initialBlockReportDelayMs = initBRDelay;
     
     heartBeatInterval = conf.getLong(DFS_HEARTBEAT_INTERVAL_KEY,
         DFS_HEARTBEAT_INTERVAL_DEFAULT) * 1000L;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index d3d98fd..d3b32da 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3976,7 +3976,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg,
       StorageReport[] reports, long cacheCapacity, long cacheUsed,
       int xceiverCount, int xmitsInProgress, int failedVolumes,
-      VolumeFailureSummary volumeFailureSummary) throws IOException {
+      VolumeFailureSummary volumeFailureSummary,
+      boolean requestFullBlockReportLease) throws IOException {
     readLock();
     try {
       //get datanode commands
@@ -3985,13 +3986,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
           nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed,
           xceiverCount, maxTransfer, failedVolumes, volumeFailureSummary);
-      
+      long blockReportLeaseId = 0;
+      if (requestFullBlockReportLease) {
+        blockReportLeaseId =  blockManager.requestBlockReportLeaseId(nodeReg);
+      }
       //create ha status
       final NNHAStatusHeartbeat haState = new NNHAStatusHeartbeat(
           haContext.getState().getServiceState(),
           getFSImage().getLastAppliedOrWrittenTxId());
 
-      return new HeartbeatResponse(cmds, haState, rollingUpgradeInfo);
+      return new HeartbeatResponse(cmds, haState, rollingUpgradeInfo,
+          blockReportLeaseId);
     } finally {
       readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 4a146ff..52aaabd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -116,6 +116,7 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerFaultInjector;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
@@ -1277,13 +1278,13 @@ class NameNodeRpcServer implements NamenodeProtocols {
   public HeartbeatResponse sendHeartbeat(DatanodeRegistration nodeReg,
       StorageReport[] report, long dnCacheCapacity, long dnCacheUsed,
       int xmitsInProgress, int xceiverCount,
-      int failedVolumes, VolumeFailureSummary volumeFailureSummary)
-      throws IOException {
+      int failedVolumes, VolumeFailureSummary volumeFailureSummary,
+      boolean requestFullBlockReportLease) throws IOException {
     checkNNStartup();
     verifyRequest(nodeReg);
     return namesystem.handleHeartbeat(nodeReg, report,
         dnCacheCapacity, dnCacheUsed, xceiverCount, xmitsInProgress,
-        failedVolumes, volumeFailureSummary);
+        failedVolumes, volumeFailureSummary, requestFullBlockReportLease);
   }
 
   @Override // DatanodeProtocol
@@ -1309,6 +1310,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
           blocks, context, (r == reports.length - 1));
       metrics.incrStorageBlockReportOps();
     }
+    BlockManagerFaultInjector.getInstance().
+        incomingBlockReportRpc(nodeReg, context);
 
     if (nn.getFSImage().isUpgradeFinalized() &&
         !namesystem.isRollingUpgrade() &&

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockReportContext.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockReportContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockReportContext.java
index d0b0282..5bcd719 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockReportContext.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockReportContext.java
@@ -31,14 +31,33 @@ import org.apache.hadoop.classification.InterfaceAudience;
  */
 @InterfaceAudience.Private
 public class BlockReportContext {
+  /**
+   * The total number of RPCs contained in the block report.
+   */
   private final int totalRpcs;
+
+  /**
+   * The index of this particular RPC.
+   */
   private final int curRpc;
+
+  /**
+   * A 64-bit ID which identifies the block report as a whole.
+   */
   private final long reportId;
 
-  public BlockReportContext(int totalRpcs, int curRpc, long reportId) {
+  /**
+   * The lease ID which this block report is using, or 0 if this block report is
+   * bypassing rate-limiting.
+   */
+  private final long leaseId;
+
+  public BlockReportContext(int totalRpcs, int curRpc,
+                            long reportId, long leaseId) {
     this.totalRpcs = totalRpcs;
     this.curRpc = curRpc;
     this.reportId = reportId;
+    this.leaseId = leaseId;
   }
 
   public int getTotalRpcs() {
@@ -52,4 +71,8 @@ public class BlockReportContext {
   public long getReportId() {
     return reportId;
   }
+
+  public long getLeaseId() {
+    return leaseId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index a3b6004..dfe0813 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -102,6 +102,8 @@ public interface DatanodeProtocol {
    * @param xceiverCount number of active transceiver threads
    * @param failedVolumes number of failed volumes
    * @param volumeFailureSummary info about volume failures
+   * @param requestFullBlockReportLease whether to request a full block
+   *                                    report lease.
    * @throws IOException on error
    */
   @Idempotent
@@ -112,7 +114,8 @@ public interface DatanodeProtocol {
                                        int xmitsInProgress,
                                        int xceiverCount,
                                        int failedVolumes,
-                                       VolumeFailureSummary volumeFailureSummary)
+                                       VolumeFailureSummary volumeFailureSummary,
+                                       boolean requestFullBlockReportLease)
       throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/HeartbeatResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/HeartbeatResponse.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/HeartbeatResponse.java
index d00179e..8d6384e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/HeartbeatResponse.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/HeartbeatResponse.java
@@ -34,12 +34,16 @@ public class HeartbeatResponse {
   private final NNHAStatusHeartbeat haStatus;
 
   private final RollingUpgradeStatus rollingUpdateStatus;
+
+  private final long fullBlockReportLeaseId;
   
   public HeartbeatResponse(DatanodeCommand[] cmds,
-      NNHAStatusHeartbeat haStatus, RollingUpgradeStatus rollingUpdateStatus) {
+      NNHAStatusHeartbeat haStatus, RollingUpgradeStatus rollingUpdateStatus,
+      long fullBlockReportLeaseId) {
     commands = cmds;
     this.haStatus = haStatus;
     this.rollingUpdateStatus = rollingUpdateStatus;
+    this.fullBlockReportLeaseId = fullBlockReportLeaseId;
   }
   
   public DatanodeCommand[] getCommands() {
@@ -53,4 +57,8 @@ public class HeartbeatResponse {
   public RollingUpgradeStatus getRollingUpdateStatus() {
     return rollingUpdateStatus;
   }
+
+  public long getFullBlockReportLeaseId() {
+    return fullBlockReportLeaseId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java
index a102c82..2f7d334 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/RegisterCommand.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 
 /**
- * A BlockCommand is an instruction to a datanode to register with the namenode.
+ * A RegisterCommand is an instruction to a datanode to register with the namenode.
  * This command can't be combined with other commands in the same response.
  * This is because after the datanode processes RegisterCommand, it will skip
  * the rest of the DatanodeCommands in the same HeartbeatResponse.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
index 3083dc9..b87e753 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -195,6 +195,7 @@ message HeartbeatRequestProto {
   optional uint64 cacheCapacity = 6 [ default = 0 ];
   optional uint64 cacheUsed = 7 [default = 0 ];
   optional VolumeFailureSummaryProto volumeFailureSummary = 8;
+  optional bool requestFullBlockReportLease = 9 [ default = false ];
 }
 
 /**
@@ -214,6 +215,7 @@ message HeartbeatResponseProto {
   repeated DatanodeCommandProto cmds = 1; // Returned commands can be null
   required NNHAStatusHeartbeatProto haStatus = 2;
   optional RollingUpgradeStatusProto rollingUpgradeStatus = 3;
+  optional uint64 fullBlockReportLeaseId = 4 [ default = 0 ];
 }
 
 /**
@@ -243,6 +245,10 @@ message BlockReportContextProto  {
 
   // The unique 64-bit ID of this block report
   required int64 id = 3;
+
+  // The block report lease ID, or 0 if we are sending without a lease to
+  // bypass rate-limiting.
+  optional uint64 leaseId = 4 [ default = 0 ];
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 7b579cb..fdb0bc8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -578,6 +578,27 @@
 </property>
 
 <property>
+  <name>dfs.namenode.max.full.block.report.leases</name>
+  <value>6</value>
+  <description>The maximum number of leases for full block reports that the
+    NameNode will issue at any given time.  This prevents the NameNode from
+    being flooded with full block reports that use up all the RPC handler
+    threads.  This number should never be more than the number of RPC handler
+    threads or less than 1.
+  </description>
+</property>
+
+<property>
+  <name>dfs.namenode.full.block.report.lease.length.ms</name>
+  <value>300000</value>
+  <description>
+    The number of milliseconds that the NameNode will wait before invalidating
+    a full block report lease.  This prevents a crashed DataNode from
+    permanently using up a full block report lease.
+  </description>
+</property>
+
+<property>
   <name>dfs.datanode.directoryscan.interval</name>
   <value>21600</value>
   <description>Interval in seconds for Datanode to scan data directories and

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java
index f0dab4c..9ead765 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestBlockListAsLongs.java
@@ -221,7 +221,7 @@ public class TestBlockListAsLongs {
     request.set(null);
     nsInfo.setCapabilities(Capability.STORAGE_BLOCK_REPORT_BUFFERS.getMask());
     nn.blockReport(reg, "pool", sbr,
-        new BlockReportContext(1, 0, System.nanoTime()));
+        new BlockReportContext(1, 0, System.nanoTime(), 0L));
     BlockReportRequestProto proto = request.get();
     assertNotNull(proto);
     assertTrue(proto.getReports(0).getBlocksList().isEmpty());
@@ -231,7 +231,7 @@ public class TestBlockListAsLongs {
     request.set(null);
     nsInfo.setCapabilities(Capability.UNKNOWN.getMask());
     nn.blockReport(reg, "pool", sbr,
-        new BlockReportContext(1, 0, System.nanoTime()));
+        new BlockReportContext(1, 0, System.nanoTime(), 0L));
     proto = request.get();
     assertNotNull(proto);
     assertFalse(proto.getReports(0).getBlocksList().isEmpty());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockReportRateLimiting.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockReportRateLimiting.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockReportRateLimiting.java
new file mode 100644
index 0000000..fc5f9e7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockReportRateLimiting.java
@@ -0,0 +1,246 @@
+/**
+ * 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.blockmanagement;
+
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_FULL_BLOCK_REPORT_LEASES;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_FULL_BLOCK_REPORT_LEASE_LENGTH_MS;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Supplier;
+import com.google.common.util.concurrent.Uninterruptibles;
+import org.apache.commons.lang.mutable.MutableObject;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class TestBlockReportRateLimiting {
+  static final Log LOG = LogFactory.getLog(TestBlockReportRateLimiting.class);
+
+  private static void setFailure(AtomicReference<String> failure,
+                                 String what) {
+    failure.compareAndSet("", what);
+    LOG.error("Test error: " + what);
+  }
+
+  @After
+  public void restoreNormalBlockManagerFaultInjector() {
+    BlockManagerFaultInjector.instance = new BlockManagerFaultInjector();
+  }
+
+  @BeforeClass
+  public static void raiseBlockManagerLogLevels() {
+    GenericTestUtils.setLogLevel(BlockManager.LOG, Level.ALL);
+    GenericTestUtils.setLogLevel(BlockReportLeaseManager.LOG, Level.ALL);
+  }
+
+  @Test(timeout=180000)
+  public void testRateLimitingDuringDataNodeStartup() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFS_NAMENODE_MAX_FULL_BLOCK_REPORT_LEASES, 1);
+    conf.setLong(DFS_NAMENODE_FULL_BLOCK_REPORT_LEASE_LENGTH_MS,
+        20L * 60L * 1000L);
+
+    final Semaphore fbrSem = new Semaphore(0);
+    final HashSet<DatanodeID> expectedFbrDns = new HashSet<>();
+    final HashSet<DatanodeID> fbrDns = new HashSet<>();
+    final AtomicReference<String> failure = new AtomicReference<String>("");
+
+    final BlockManagerFaultInjector injector = new BlockManagerFaultInjector() {
+      private int numLeases = 0;
+
+      @Override
+      public void incomingBlockReportRpc(DatanodeID nodeID,
+                    BlockReportContext context) throws IOException {
+        LOG.info("Incoming full block report from " + nodeID +
+            ".  Lease ID = 0x" + Long.toHexString(context.getLeaseId()));
+        if (context.getLeaseId() == 0) {
+          setFailure(failure, "Got unexpected rate-limiting-" +
+              "bypassing full block report RPC from " + nodeID);
+        }
+        fbrSem.acquireUninterruptibly();
+        synchronized (this) {
+          fbrDns.add(nodeID);
+          if (!expectedFbrDns.remove(nodeID)) {
+            setFailure(failure, "Got unexpected full block report " +
+                "RPC from " + nodeID + ".  expectedFbrDns = " +
+                Joiner.on(", ").join(expectedFbrDns));
+          }
+          LOG.info("Proceeding with full block report from " +
+              nodeID + ".  Lease ID = 0x" +
+              Long.toHexString(context.getLeaseId()));
+        }
+      }
+
+      @Override
+      public void requestBlockReportLease(DatanodeDescriptor node,
+                                          long leaseId) {
+        if (leaseId == 0) {
+          return;
+        }
+        synchronized (this) {
+          numLeases++;
+          expectedFbrDns.add(node);
+          LOG.info("requestBlockReportLease(node=" + node +
+              ", leaseId=0x" + Long.toHexString(leaseId) + ").  " +
+              "expectedFbrDns = " +  Joiner.on(", ").join(expectedFbrDns));
+          if (numLeases > 1) {
+            setFailure(failure, "More than 1 lease was issued at once.");
+          }
+        }
+      }
+
+      @Override
+      public void removeBlockReportLease(DatanodeDescriptor node, long leaseId) {
+        LOG.info("removeBlockReportLease(node=" + node +
+                 ", leaseId=0x" + Long.toHexString(leaseId) + ")");
+        synchronized (this) {
+          numLeases--;
+        }
+      }
+    };
+    BlockManagerFaultInjector.instance = injector;
+
+    final int NUM_DATANODES = 5;
+    MiniDFSCluster cluster =
+        new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES).build();
+    cluster.waitActive();
+    for (int n = 1; n <= NUM_DATANODES; n++) {
+      LOG.info("Waiting for " + n + " datanode(s) to report in.");
+      fbrSem.release();
+      Uninterruptibles.sleepUninterruptibly(20, TimeUnit.MILLISECONDS);
+      final int currentN = n;
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          synchronized (injector) {
+            if (fbrDns.size() > currentN) {
+              setFailure(failure, "Expected at most " + currentN +
+                  " datanodes to have sent a block report, but actually " +
+                  fbrDns.size() + " have.");
+            }
+            return (fbrDns.size() >= currentN);
+          }
+        }
+      }, 25, 50000);
+    }
+    cluster.shutdown();
+    Assert.assertEquals("", failure.get());
+  }
+
+  /**
+   * Start a 2-node cluster with only one block report lease.  When the
+   * first datanode gets a lease, kill it.  Then wait for the lease to
+   * expire, and the second datanode to send a full block report.
+   */
+  @Test(timeout=180000)
+  public void testLeaseExpiration() throws Exception {
+    Configuration conf = new Configuration();
+    conf.setInt(DFS_NAMENODE_MAX_FULL_BLOCK_REPORT_LEASES, 1);
+    conf.setLong(DFS_NAMENODE_FULL_BLOCK_REPORT_LEASE_LENGTH_MS, 100L);
+
+    final Semaphore gotFbrSem = new Semaphore(0);
+    final AtomicReference<String> failure = new AtomicReference<String>("");
+    final AtomicReference<MiniDFSCluster> cluster =
+        new AtomicReference<>(null);
+    final BlockingQueue<Integer> datanodeToStop =
+        new ArrayBlockingQueue<Integer>(1);
+    final BlockManagerFaultInjector injector = new BlockManagerFaultInjector() {
+      private String uuidToStop = "";
+
+      @Override
+      public void incomingBlockReportRpc(DatanodeID nodeID,
+                BlockReportContext context) throws IOException {
+        if (context.getLeaseId() == 0) {
+          setFailure(failure, "Got unexpected rate-limiting-" +
+              "bypassing full block report RPC from " + nodeID);
+        }
+        synchronized (this) {
+          if (uuidToStop.equals(nodeID.getDatanodeUuid())) {
+            throw new IOException("Injecting failure into block " +
+                "report RPC for " + nodeID);
+          }
+        }
+        gotFbrSem.release();
+      }
+
+      @Override
+      public void requestBlockReportLease(DatanodeDescriptor node,
+                                          long leaseId) {
+        if (leaseId == 0) {
+          return;
+        }
+        synchronized (this) {
+          if (uuidToStop.isEmpty()) {
+            MiniDFSCluster cl;
+            do {
+              cl = cluster.get();
+            } while (cl == null);
+            int datanodeIndexToStop = getDatanodeIndex(cl, node);
+            uuidToStop = node.getDatanodeUuid();
+            datanodeToStop.add(Integer.valueOf(datanodeIndexToStop));
+          }
+        }
+      }
+
+      private int getDatanodeIndex(MiniDFSCluster cl,
+                                   DatanodeDescriptor node) {
+        List<DataNode> datanodes = cl.getDataNodes();
+        for (int i = 0; i < datanodes.size(); i++) {
+          DataNode datanode = datanodes.get(i);
+          if (datanode.getDatanodeUuid().equals(node.getDatanodeUuid())) {
+            return i;
+          }
+        }
+        throw new RuntimeException("Failed to find UUID " +
+            node.getDatanodeUuid() + " in the list of datanodes.");
+      }
+
+      @Override
+      public void removeBlockReportLease(DatanodeDescriptor node, long leaseId) {
+      }
+    };
+    BlockManagerFaultInjector.instance = injector;
+    cluster.set(new MiniDFSCluster.Builder(conf).numDataNodes(2).build());
+    cluster.get().waitActive();
+    int datanodeIndexToStop = datanodeToStop.take();
+    cluster.get().stopDataNode(datanodeIndexToStop);
+    gotFbrSem.acquire();
+    cluster.get().shutdown();
+    Assert.assertEquals("", failure.get());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
index bf167a5..39bd5d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java
@@ -59,6 +59,15 @@ public class TestDatanodeManager {
   //The number of times the registration / removal of nodes should happen
   final int NUM_ITERATIONS = 500;
 
+  private static DatanodeManager mockDatanodeManager(
+      FSNamesystem fsn, Configuration conf) throws IOException {
+    BlockManager bm = Mockito.mock(BlockManager.class);
+    BlockReportLeaseManager blm = new BlockReportLeaseManager(conf);
+    Mockito.when(bm.getBlockReportLeaseManager()).thenReturn(blm);
+    DatanodeManager dm = new DatanodeManager(bm, fsn, conf);
+    return dm;
+  }
+
   /**
    * This test sends a random sequence of node registrations and node removals
    * to the DatanodeManager (of nodes with different IDs and versions), and
@@ -70,8 +79,7 @@ public class TestDatanodeManager {
     //Create the DatanodeManager which will be tested
     FSNamesystem fsn = Mockito.mock(FSNamesystem.class);
     Mockito.when(fsn.hasWriteLock()).thenReturn(true);
-    DatanodeManager dm = new DatanodeManager(Mockito.mock(BlockManager.class),
-      fsn, new Configuration());
+    DatanodeManager dm = mockDatanodeManager(fsn, new Configuration());
 
     //Seed the RNG with a known value so test failures are easier to reproduce
     Random rng = new Random();
@@ -183,9 +191,8 @@ public class TestDatanodeManager {
         TestDatanodeManager.MyResolver.class, DNSToSwitchMapping.class);
     
     //create DatanodeManager
-    DatanodeManager dm = new DatanodeManager(Mockito.mock(BlockManager.class),
-        fsn, conf);
-    
+    DatanodeManager dm = mockDatanodeManager(fsn, conf);
+
     //storageID to register.
     String storageID = "someStorageID-123";
     
@@ -258,7 +265,6 @@ public class TestDatanodeManager {
     HelperFunction("/"+ Shell.appendScriptExtension("topology-broken-script"));
   }
 
-
   /**
    * Helper function that tests the DatanodeManagers SortedBlock function
    * we invoke this function with and without topology scripts
@@ -281,8 +287,7 @@ public class TestDatanodeManager {
       conf.set(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY,
         resourcePath.toString());
     }
-    DatanodeManager dm = new DatanodeManager(Mockito.mock(BlockManager.class),
-      fsn, conf);
+    DatanodeManager dm = mockDatanodeManager(fsn, conf);
 
     // register 5 datanodes, each with different storage ID and type
     DatanodeInfo[] locs = new DatanodeInfo[5];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
index d73f63e..cea6865 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
@@ -115,7 +115,7 @@ public class TestNameNodePrunesMissingStorages {
       // Stop the DataNode and send fake heartbeat with missing storage.
       cluster.stopDataNode(0);
       cluster.getNameNodeRpc().sendHeartbeat(dnReg, prunedReports, 0L, 0L, 0, 0,
-          0, null);
+          0, null, true);
 
       // Check that the missing storage was pruned.
       assertThat(dnDescriptor.getStorageInfos().length, is(expectedStoragesAfterTest));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
index 64cc78b..f970b3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
@@ -28,6 +28,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.logging.Log;
@@ -143,7 +144,8 @@ public class TestBPOfferService {
           Mockito.anyInt(),
           Mockito.anyInt(),
           Mockito.anyInt(),
-          Mockito.any(VolumeFailureSummary.class));
+          Mockito.any(VolumeFailureSummary.class),
+          Mockito.anyBoolean());
     mockHaStatuses[nnIdx] = new NNHAStatusHeartbeat(HAServiceState.STANDBY, 0);
     return mock;
   }
@@ -164,7 +166,8 @@ public class TestBPOfferService {
     public HeartbeatResponse answer(InvocationOnMock invocation) throws Throwable {
       heartbeatCounts[nnIdx]++;
       return new HeartbeatResponse(new DatanodeCommand[0],
-          mockHaStatuses[nnIdx], null);
+          mockHaStatuses[nnIdx], null,
+          ThreadLocalRandom.current().nextLong() | 1L);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
index c65ef85..27d1cea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockHasMultipleReplicasOnSameDN.java
@@ -126,7 +126,7 @@ public class TestBlockHasMultipleReplicasOnSameDN {
 
     // Should not assert!
     cluster.getNameNodeRpc().blockReport(dnReg, bpid, reports,
-        new BlockReportContext(1, 0, System.nanoTime()));
+        new BlockReportContext(1, 0, System.nanoTime(), 0L));
 
     // Get the block locations once again.
     locatedBlocks = client.getLocatedBlocks(filename, 0, BLOCK_SIZE * NUM_BLOCKS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
index f91c0bc..7552e10 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
@@ -39,6 +39,7 @@ import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
@@ -162,11 +163,12 @@ public class TestBlockRecovery {
             Mockito.anyInt(),
             Mockito.anyInt(),
             Mockito.anyInt(),
-            Mockito.any(VolumeFailureSummary.class)))
+            Mockito.any(VolumeFailureSummary.class),
+            Mockito.anyBoolean()))
         .thenReturn(new HeartbeatResponse(
             new DatanodeCommand[0],
             new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1),
-            null));
+            null, ThreadLocalRandom.current().nextLong() | 1L));
 
     dn = new DataNode(conf, locations, null) {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java
index 0d7484c..b9b6512 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java
@@ -57,7 +57,7 @@ public class TestBpServiceActorScheduler {
     for (final long now : getTimestamps()) {
       Scheduler scheduler = makeMockScheduler(now);
       assertTrue(scheduler.isHeartbeatDue(now));
-      assertTrue(scheduler.isBlockReportDue());
+      assertTrue(scheduler.isBlockReportDue(scheduler.monotonicNow()));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
index bf80887..e784c7a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeProtocolRetryPolicy.java
@@ -27,6 +27,7 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
+import java.util.concurrent.ThreadLocalRandom;
 
 import com.google.common.base.Supplier;
 import org.apache.commons.logging.Log;
@@ -199,13 +200,13 @@ public class TestDatanodeProtocolRetryPolicy {
           heartbeatResponse = new HeartbeatResponse(
               new DatanodeCommand[]{RegisterCommand.REGISTER},
               new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1),
-              null);
+              null, ThreadLocalRandom.current().nextLong() | 1L);
         } else {
           LOG.info("mockito heartbeatResponse " + i);
           heartbeatResponse = new HeartbeatResponse(
               new DatanodeCommand[0],
               new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1),
-              null);
+              null, ThreadLocalRandom.current().nextLong() | 1L);
         }
         return heartbeatResponse;
       }
@@ -217,7 +218,8 @@ public class TestDatanodeProtocolRetryPolicy {
            Mockito.anyInt(),
            Mockito.anyInt(),
            Mockito.anyInt(),
-           Mockito.any(VolumeFailureSummary.class));
+           Mockito.any(VolumeFailureSummary.class),
+           Mockito.anyBoolean());
 
     dn = new DataNode(conf, locations, null) {
       @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
index 58932fb..cb4022e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java
@@ -21,6 +21,7 @@ import static org.apache.hadoop.test.MetricsAsserts.getMetrics;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyBoolean;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Matchers.anyLong;
 import static org.mockito.Mockito.doReturn;
@@ -31,6 +32,7 @@ import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.HashSet;
 import java.util.Set;
+import java.util.concurrent.ThreadLocalRandom;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -159,11 +161,14 @@ public class TestFsDatasetCache {
       throws IOException {
     NNHAStatusHeartbeat ha = new NNHAStatusHeartbeat(HAServiceState.ACTIVE,
         fsImage.getLastAppliedOrWrittenTxId());
-    HeartbeatResponse response = new HeartbeatResponse(cmds, ha, null);
+    HeartbeatResponse response =
+        new HeartbeatResponse(cmds, ha, null,
+            ThreadLocalRandom.current().nextLong() | 1L);
     doReturn(response).when(spyNN).sendHeartbeat(
         (DatanodeRegistration) any(),
         (StorageReport[]) any(), anyLong(), anyLong(),
-        anyInt(), anyInt(), anyInt(), (VolumeFailureSummary) any());
+        anyInt(), anyInt(), anyInt(), (VolumeFailureSummary) any(),
+        anyBoolean());
   }
 
   private static DatanodeCommand[] cacheBlock(HdfsBlockLocation loc) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestNNHandlesBlockReportPerStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestNNHandlesBlockReportPerStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestNNHandlesBlockReportPerStorage.java
index b150b0d..67bbefe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestNNHandlesBlockReportPerStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestNNHandlesBlockReportPerStorage.java
@@ -40,7 +40,7 @@ public class TestNNHandlesBlockReportPerStorage extends BlockReportTestBase {
       LOG.info("Sending block report for storage " + report.getStorage().getStorageID());
       StorageBlockReport[] singletonReport = { report };
       cluster.getNameNodeRpc().blockReport(dnR, poolId, singletonReport,
-          new BlockReportContext(reports.length, i, System.nanoTime()));
+          new BlockReportContext(reports.length, i, System.nanoTime(), 0L));
       i++;
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestNNHandlesCombinedBlockReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestNNHandlesCombinedBlockReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestNNHandlesCombinedBlockReport.java
index dca3c88..fd19ba6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestNNHandlesCombinedBlockReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestNNHandlesCombinedBlockReport.java
@@ -36,6 +36,6 @@ public class TestNNHandlesCombinedBlockReport extends BlockReportTestBase {
                                   StorageBlockReport[] reports) throws IOException {
     LOG.info("Sending combined block reports for " + dnR);
     cluster.getNameNodeRpc().blockReport(dnR, poolId, reports,
-        new BlockReportContext(1, 0, System.nanoTime()));
+        new BlockReportContext(1, 0, System.nanoTime(), 0L));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
index ecb28dc..a6032c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestStorageReport.java
@@ -106,7 +106,7 @@ public class TestStorageReport {
         any(DatanodeRegistration.class),
         captor.capture(),
         anyLong(), anyLong(), anyInt(), anyInt(), anyInt(),
-        Mockito.any(VolumeFailureSummary.class));
+        Mockito.any(VolumeFailureSummary.class), Mockito.anyBoolean());
 
     StorageReport[] reports = captor.getValue();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
index 2964f9a..39894b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NNThroughputBenchmark.java
@@ -968,7 +968,7 @@ public class NNThroughputBenchmark implements Tool {
           new StorageBlockReport(storage, BlockListAsLongs.EMPTY)
       };
       dataNodeProto.blockReport(dnRegistration, bpid, reports,
-              new BlockReportContext(1, 0, System.nanoTime()));
+              new BlockReportContext(1, 0, System.nanoTime(), 0L));
     }
 
     /**
@@ -981,7 +981,7 @@ public class NNThroughputBenchmark implements Tool {
       StorageReport[] rep = { new StorageReport(storage, false,
           DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) };
       DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration, rep,
-          0L, 0L, 0, 0, 0, null).getCommands();
+          0L, 0L, 0, 0, 0, null, true).getCommands();
       if(cmds != null) {
         for (DatanodeCommand cmd : cmds ) {
           if(LOG.isDebugEnabled()) {
@@ -1030,7 +1030,7 @@ public class NNThroughputBenchmark implements Tool {
       StorageReport[] rep = { new StorageReport(storage,
           false, DF_CAPACITY, DF_USED, DF_CAPACITY - DF_USED, DF_USED) };
       DatanodeCommand[] cmds = dataNodeProto.sendHeartbeat(dnRegistration,
-          rep, 0L, 0L, 0, 0, 0, null).getCommands();
+          rep, 0L, 0L, 0, 0, 0, null, true).getCommands();
       if (cmds != null) {
         for (DatanodeCommand cmd : cmds) {
           if (cmd.getAction() == DatanodeProtocol.DNA_TRANSFER) {
@@ -1213,7 +1213,7 @@ public class NNThroughputBenchmark implements Tool {
       StorageBlockReport[] report = { new StorageBlockReport(
           dn.storage, dn.getBlockReportList()) };
       dataNodeProto.blockReport(dn.dnRegistration, bpid, report,
-          new BlockReportContext(1, 0, System.nanoTime()));
+          new BlockReportContext(1, 0, System.nanoTime(), 0L));
       long end = Time.now();
       return end-start;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f7460011/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
index 4ca5eda..b314584 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/NameNodeAdapter.java
@@ -117,7 +117,7 @@ public class NameNodeAdapter {
       DatanodeDescriptor dd, FSNamesystem namesystem) throws IOException {
     return namesystem.handleHeartbeat(nodeReg,
         BlockManagerTestUtil.getStorageReportsForDatanode(dd),
-        dd.getCacheCapacity(), dd.getCacheRemaining(), 0, 0, 0, null);
+        dd.getCacheCapacity(), dd.getCacheRemaining(), 0, 0, 0, null, true);
   }
 
   public static boolean setReplication(final FSNamesystem ns,


[28/50] [abbrv] hadoop git commit: HDFS-8548. Minicluster throws NPE on shutdown. Contributed by surendra singh lilhore.

Posted by zj...@apache.org.
HDFS-8548. Minicluster throws NPE on shutdown. Contributed by surendra singh lilhore.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/816ab879
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/816ab879
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/816ab879

Branch: refs/heads/YARN-2928
Commit: 816ab879661b6ed580b21f0b47771bec5115451f
Parents: b509768
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Jun 16 13:52:50 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:18:58 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                    | 3 +++
 .../apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java   | 6 +++++-
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/816ab879/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index b921f2c..c8c36e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -900,6 +900,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-8592. SafeModeException never get unwrapped. (wheat9)
 
+    HDFS-8548. Minicluster throws NPE on shutdown.
+    (surendra singh lilhore via xyao)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/816ab879/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
index 9eb1059..0dbf485 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
@@ -198,7 +198,11 @@ class BlocksMap {
   }
 
   int size() {
-    return blocks.size();
+    if (blocks != null) {
+      return blocks.size();
+    } else {
+      return 0;
+    }
   }
 
   Iterable<BlockInfo> getBlocks() {


[41/50] [abbrv] hadoop git commit: Update CHANGES.txt for HDFS-8238.

Posted by zj...@apache.org.
Update CHANGES.txt for HDFS-8238.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c10bcc06
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c10bcc06
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c10bcc06

Branch: refs/heads/YARN-2928
Commit: c10bcc0618eb502eeabb33a0fc6ae49aac1b01dc
Parents: eb2ccad
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Jun 17 16:19:45 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:01 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c10bcc06/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a01446a..6ef405b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -644,6 +644,9 @@ Release 2.8.0 - UNRELEASED
     of Block in UnderReplicatedBlocks and PendingReplicationBlocks).
     (Zhe Zhang via wang)
 
+    HDFS-8238. Move ClientProtocol to the hdfs-client.
+    (Takanobu Asanuma via wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[33/50] [abbrv] hadoop git commit: HDFS-8551. Fix hdfs datanode CLI usage message. Contributed by Brahma Reddy Battula.

Posted by zj...@apache.org.
HDFS-8551. Fix hdfs datanode CLI usage message. Contributed by Brahma Reddy Battula.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/984e8bee
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/984e8bee
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/984e8bee

Branch: refs/heads/YARN-2928
Commit: 984e8bee01e71e97840b08e1bb782ceb921d260e
Parents: 76dc2f5
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Jun 16 14:12:45 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:18:59 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 3 +++
 .../java/org/apache/hadoop/hdfs/server/datanode/DataNode.java    | 4 +++-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/984e8bee/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c8c36e6..2d4c062 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -903,6 +903,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8548. Minicluster throws NPE on shutdown.
     (surendra singh lilhore via xyao)
 
+    HDFS-8551. Fix hdfs datanode CLI usage message.
+    (Brahma Reddy Battula via xyao)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/984e8bee/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index 3bd131e..74acf81 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -262,9 +262,11 @@ public class DataNode extends ReconfigurableBase
     LogFactory.getLog(DataNode.class.getName() + ".clienttrace");
   
   private static final String USAGE =
-      "Usage: java DataNode [-regular | -rollback]\n" +
+      "Usage: hdfs datanode [-regular | -rollback | -rollingupgrade rollback" +
+      " ]\n" +
       "    -regular                 : Normal DataNode startup (default).\n" +
       "    -rollback                : Rollback a standard or rolling upgrade.\n" +
+      "    -rollingupgrade rollback : Rollback a rolling upgrade operation.\n" +
       "  Refer to HDFS documentation for the difference between standard\n" +
       "  and rolling upgrades.";
 


[34/50] [abbrv] hadoop git commit: HADOOP-12076. Incomplete Cache Mechanism in CredentialProvider API. Contributed by Larry McCay.

Posted by zj...@apache.org.
HADOOP-12076. Incomplete Cache Mechanism in CredentialProvider API. Contributed by Larry McCay.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3146ab13
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3146ab13
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3146ab13

Branch: refs/heads/YARN-2928
Commit: 3146ab13fe4d9f5288fa4da9a3a2a53fc965d01f
Parents: 984e8be
Author: cnauroth <cn...@apache.org>
Authored: Tue Jun 16 14:44:03 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:18:59 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt      |  3 +++
 .../security/alias/AbstractJavaKeyStoreProvider.java | 15 +--------------
 2 files changed, 4 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3146ab13/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 9a16a7d..67e9c76 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -861,6 +861,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12095. org.apache.hadoop.fs.shell.TestCount fails.
     (Brahma Reddy Battula via aajisaka)
 
+    HADOOP-12076. Incomplete Cache Mechanism in CredentialProvider API.
+    (Larry McCay via cnauroth)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3146ab13/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java
index 76b8cd5..9656261 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/AbstractJavaKeyStoreProvider.java
@@ -41,9 +41,7 @@ import java.security.UnrecoverableKeyException;
 import java.security.cert.CertificateException;
 import java.util.ArrayList;
 import java.util.Enumeration;
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -174,13 +172,6 @@ public abstract class AbstractJavaKeyStoreProvider extends CredentialProvider {
     return keyStore;
   }
 
-  public Map<String, CredentialEntry> getCache() {
-    return cache;
-  }
-
-  private final Map<String, CredentialEntry> cache =
-      new HashMap<String, CredentialEntry>();
-
   protected final String getPathAsString() {
     return getPath().toString();
   }
@@ -213,9 +204,6 @@ public abstract class AbstractJavaKeyStoreProvider extends CredentialProvider {
     try {
       SecretKeySpec key = null;
       try {
-        if (cache.containsKey(alias)) {
-          return cache.get(alias);
-        }
         if (!keyStore.containsAlias(alias)) {
           return null;
         }
@@ -269,7 +257,7 @@ public abstract class AbstractJavaKeyStoreProvider extends CredentialProvider {
       throws IOException {
     writeLock.lock();
     try {
-      if (keyStore.containsAlias(alias) || cache.containsKey(alias)) {
+      if (keyStore.containsAlias(alias)) {
         throw new IOException("Credential " + alias + " already exists in "
             + this);
       }
@@ -296,7 +284,6 @@ public abstract class AbstractJavaKeyStoreProvider extends CredentialProvider {
       } catch (KeyStoreException e) {
         throw new IOException("Problem removing " + name + " from " + this, e);
       }
-      cache.remove(name);
       changed = true;
     } finally {
       writeLock.unlock();


[40/50] [abbrv] hadoop git commit: HDFS-8615. Correct HTTP method in WebHDFS document. Contributed by Brahma Reddy Battula.

Posted by zj...@apache.org.
HDFS-8615. Correct HTTP method in WebHDFS document. Contributed by Brahma Reddy Battula.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c1816803
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c1816803
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c1816803

Branch: refs/heads/YARN-2928
Commit: c1816803b6ebe64fe6651f18f83c60ff9d2a251e
Parents: 59171b4
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Jun 17 17:13:02 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:01 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt             |  3 +++
 .../hadoop-hdfs/src/site/markdown/WebHDFS.md            | 12 ++++++------
 2 files changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1816803/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index ef3530f3..8f563de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -924,6 +924,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8551. Fix hdfs datanode CLI usage message.
     (Brahma Reddy Battula via xyao)
 
+    HDFS-8615. Correct HTTP method in WebHDFS document.
+    (Brahma Reddy Battula via aajisaka)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c1816803/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
index d0231a3..e8f5fee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md
@@ -691,7 +691,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).setAcl
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETACLSTATUS"
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETACLSTATUS"
 
     The client receives a response with a [`AclStatus` JSON object](#ACL_Status_JSON_Schema):
 
@@ -718,7 +718,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getAclSta
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=CHECKACCESS
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=CHECKACCESS
                                       &fsaction=<FSACTION>
 
     The client receives a response with zero content length:
@@ -764,7 +764,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).removeXAt
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
                                       &xattr.name=<XATTRNAME>&encoding=<ENCODING>"
 
     The client receives a response with a [`XAttrs` JSON object](#XAttrs_JSON_Schema):
@@ -788,7 +788,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getXAttr
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
                                       &xattr.name=<XATTRNAME1>&xattr.name=<XATTRNAME2>
                                       &encoding=<ENCODING>"
 
@@ -817,7 +817,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getXAttrs
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETXATTRS
                                       &encoding=<ENCODING>"
 
     The client receives a response with a [`XAttrs` JSON object](#XAttrs_JSON_Schema):
@@ -849,7 +849,7 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getXAttrs
 
 * Submit a HTTP GET request.
 
-        curl -i -X PUT "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTXATTRS"
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=LISTXATTRS"
 
     The client receives a response with a [`XAttrNames` JSON object](#XAttrNames_JSON_Schema):
 


[44/50] [abbrv] hadoop git commit: HDFS-8589. Fix unused imports in BPServiceActor and BlockReportLeaseManager (cmccabe)

Posted by zj...@apache.org.
HDFS-8589. Fix unused imports in BPServiceActor and BlockReportLeaseManager (cmccabe)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/59171b44
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/59171b44
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/59171b44

Branch: refs/heads/YARN-2928
Commit: 59171b44df5ce8e7b81d2500534db47d4cc06e87
Parents: 563aa16
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jun 17 17:01:42 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:01 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hdfs/server/blockmanagement/BlockReportLeaseManager.java      | 1 -
 .../org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java    | 1 -
 3 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/59171b44/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 6dfcd18..ef3530f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -650,6 +650,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8446. Separate safemode related operations in GetBlockLocations().
     (wheat9)
 
+    HDFS-8589. Fix unused imports in BPServiceActor and BlockReportLeaseManager
+    (cmccabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/59171b44/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java
index cd037f5..7db05c7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockReportLeaseManager.java
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/59171b44/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
index ea1abbd..f84dd99 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
@@ -29,7 +29,6 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.base.Joiner;


[46/50] [abbrv] hadoop git commit: YARN-3824. Fix two minor nits in member variable properties of YarnConfiguration. Contributed by Ray Chiang.

Posted by zj...@apache.org.
YARN-3824. Fix two minor nits in member variable properties of
YarnConfiguration. Contributed by Ray Chiang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d6e632a7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d6e632a7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d6e632a7

Branch: refs/heads/YARN-2928
Commit: d6e632a7c5c90963c7a3852c7be808150361cc55
Parents: e922a27
Author: Devaraj K <de...@apache.org>
Authored: Thu Jun 18 16:44:08 2015 +0530
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:02 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                  | 3 +++
 .../main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6e632a7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6bf0620..a4ffa03 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -636,6 +636,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3617. Fix WindowsResourceCalculatorPlugin.getCpuFrequency()
     returning always -1. (J.Andreina via devaraj)
 
+    YARN-3824. Fix two minor nits in member variable properties
+    of YarnConfiguration. (Ray Chiang via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6e632a7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 9011a5a..23e6b11 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -279,7 +279,7 @@ public class YarnConfiguration extends Configuration {
       + "intermediate-data-encryption.enable";
 
   @Private
-  public static final Boolean DEFAULT_YARN_INTERMEDIATE_DATA_ENCRYPTION = false;
+  public static final boolean DEFAULT_YARN_INTERMEDIATE_DATA_ENCRYPTION = false;
 
   /** The address of the RM admin interface.*/
   public static final String RM_ADMIN_ADDRESS = 
@@ -760,7 +760,7 @@ public class YarnConfiguration extends Configuration {
 
   public static final String RM_PROXY_USER_PRIVILEGES_ENABLED = RM_PREFIX
       + "proxy-user-privileges.enabled";
-  public static boolean DEFAULT_RM_PROXY_USER_PRIVILEGES_ENABLED = false;
+  public static final boolean DEFAULT_RM_PROXY_USER_PRIVILEGES_ENABLED = false;
 
   /**
    * How many diagnostics/failure messages can be saved in RM for


[12/50] [abbrv] hadoop git commit: HDFS-8572. DN always uses HTTP/localhost@REALM principals in SPNEGO. Contributed by Haohui Mai.

Posted by zj...@apache.org.
HDFS-8572. DN always uses HTTP/localhost@REALM principals in SPNEGO. Contributed by Haohui Mai.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d6ff0e81
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d6ff0e81
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d6ff0e81

Branch: refs/heads/YARN-2928
Commit: d6ff0e81d6cf8ecd256be526efcd6538e60f17bf
Parents: bc9d48a
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Jun 11 18:53:29 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:05 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hadoop/hdfs/server/datanode/DataNode.java   | 36 ++------------
 .../server/datanode/web/DatanodeHttpServer.java | 52 ++++++++++++++++++--
 3 files changed, 55 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6ff0e81/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 033451e..51a0897 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -996,6 +996,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8583. Document that NFS gateway does not work with rpcbind
     on SLES 11. (Arpit Agarwal)
 
+    HDFS-8572. DN always uses HTTP/localhost@REALM principals in SPNEGO.
+    (wheat9)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6ff0e81/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index f73eb66..ed2925b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
+
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
@@ -148,7 +148,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
-import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
@@ -163,7 +162,6 @@ import org.apache.hadoop.hdfs.server.protocol.InterDatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReplicaRecoveryInfo;
 import org.apache.hadoop.http.HttpConfig;
-import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.ReadaheadPool;
 import org.apache.hadoop.io.nativeio.NativeIO;
@@ -180,7 +178,6 @@ import org.apache.hadoop.security.SaslPropertiesResolver;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
-import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.tracing.SpanReceiverHost;
@@ -299,7 +296,6 @@ public class DataNode extends ReconfigurableBase
   private volatile boolean heartbeatsDisabledForTests = false;
   private DataStorage storage = null;
 
-  private HttpServer2 infoServer = null;
   private DatanodeHttpServer httpServer = null;
   private int infoPort;
   private int infoSecurePort;
@@ -761,29 +757,12 @@ public class DataNode extends ReconfigurableBase
    */
   private void startInfoServer(Configuration conf)
     throws IOException {
-    Configuration confForInfoServer = new Configuration(conf);
-    confForInfoServer.setInt(HttpServer2.HTTP_MAX_THREADS, 10);
-    HttpServer2.Builder builder = new HttpServer2.Builder()
-      .setName("datanode")
-      .setConf(conf).setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")))
-      .addEndpoint(URI.create("http://localhost:0"))
-      .setFindPort(true);
-
-    this.infoServer = builder.build();
-
-    this.infoServer.setAttribute("datanode", this);
-    this.infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
-    this.infoServer.addServlet(null, "/blockScannerReport",
-                               BlockScanner.Servlet.class);
-
-    this.infoServer.start();
-    InetSocketAddress jettyAddr = infoServer.getConnectorAddress(0);
-
     // SecureDataNodeStarter will bind the privileged port to the channel if
     // the DN is started by JSVC, pass it along.
     ServerSocketChannel httpServerChannel = secureResources != null ?
-      secureResources.getHttpServerChannel() : null;
-    this.httpServer = new DatanodeHttpServer(conf, jettyAddr, httpServerChannel);
+        secureResources.getHttpServerChannel() : null;
+
+    this.httpServer = new DatanodeHttpServer(conf, httpServerChannel);
     httpServer.start();
     if (httpServer.getHttpAddress() != null) {
       infoPort = httpServer.getHttpAddress().getPort();
@@ -1715,13 +1694,6 @@ public class DataNode extends ReconfigurableBase
     shutdownPeriodicScanners();
 
     // Stop the web server
-    if (infoServer != null) {
-      try {
-        infoServer.stop();
-      } catch (Exception e) {
-        LOG.warn("Exception shutting down DataNode", e);
-      }
-    }
     if (httpServer != null) {
       try {
         httpServer.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d6ff0e81/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
index 613d3ce..e9ad92f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.datanode.web;
 
-import io.netty.channel.ChannelFactory;
 import io.netty.bootstrap.ServerBootstrap;
+import io.netty.channel.ChannelFactory;
 import io.netty.channel.ChannelFuture;
 import io.netty.channel.ChannelInitializer;
 import io.netty.channel.ChannelPipeline;
@@ -35,9 +35,13 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
+import org.apache.hadoop.hdfs.server.datanode.BlockScanner;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.http.HttpConfig;
+import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.ssl.SSLFactory;
 
 import java.io.Closeable;
@@ -46,13 +50,17 @@ import java.net.BindException;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
 import java.net.SocketException;
+import java.net.URI;
 import java.nio.channels.ServerSocketChannel;
 import java.security.GeneralSecurityException;
 
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HTTP_ADDRESS_KEY;
 
 public class DatanodeHttpServer implements Closeable {
+  private final HttpServer2 infoServer;
   private final EventLoopGroup bossGroup;
   private final EventLoopGroup workerGroup;
   private final ServerSocketChannel externalHttpChannel;
@@ -66,10 +74,31 @@ public class DatanodeHttpServer implements Closeable {
 
   static final Log LOG = LogFactory.getLog(DatanodeHttpServer.class);
 
-  public DatanodeHttpServer(final Configuration conf, final InetSocketAddress
-    jettyAddr, final ServerSocketChannel externalHttpChannel)
+  public DatanodeHttpServer(final Configuration conf,
+      final ServerSocketChannel externalHttpChannel)
     throws IOException {
     this.conf = conf;
+
+    Configuration confForInfoServer = new Configuration(conf);
+    confForInfoServer.setInt(HttpServer2.HTTP_MAX_THREADS, 10);
+    HttpServer2.Builder builder = new HttpServer2.Builder()
+        .setName("datanode")
+        .setConf(confForInfoServer)
+        .setACL(new AccessControlList(conf.get(DFS_ADMIN, " ")))
+        .hostName(getHostnameForSpnegoPrincipal(confForInfoServer))
+        .addEndpoint(URI.create("http://localhost:0"))
+        .setFindPort(true);
+
+    this.infoServer = builder.build();
+
+    this.infoServer.setAttribute("datanode", this);
+    this.infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
+    this.infoServer.addServlet(null, "/blockScannerReport",
+                               BlockScanner.Servlet.class);
+
+    this.infoServer.start();
+    final InetSocketAddress jettyAddr = infoServer.getConnectorAddress(0);
+
     this.confForCreate = new Configuration(conf);
     confForCreate.set(FsPermission.UMASK_LABEL, "000");
 
@@ -191,5 +220,20 @@ public class DatanodeHttpServer implements Closeable {
     if (externalHttpChannel != null) {
       externalHttpChannel.close();
     }
+    try {
+      infoServer.stop();
+    } catch (Exception e) {
+      throw new IOException(e);
+    }
+  }
+
+  private static String getHostnameForSpnegoPrincipal(Configuration conf) {
+    String addr = conf.getTrimmed(DFS_DATANODE_HTTP_ADDRESS_KEY, null);
+    if (addr == null) {
+      addr = conf.getTrimmed(DFS_DATANODE_HTTPS_ADDRESS_KEY,
+                             DFS_DATANODE_HTTPS_ADDRESS_DEFAULT);
+    }
+    InetSocketAddress inetSocker = NetUtils.createSocketAddr(addr);
+    return inetSocker.getHostString();
   }
-}
+}
\ No newline at end of file


[35/50] [abbrv] hadoop git commit: YARN-3617. Fix WindowsResourceCalculatorPlugin.getCpuFrequency() returning always -1. Contributed by J.Andreina.

Posted by zj...@apache.org.
YARN-3617. Fix WindowsResourceCalculatorPlugin.getCpuFrequency() returning
always -1. Contributed by J.Andreina.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/70281247
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/70281247
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/70281247

Branch: refs/heads/YARN-2928
Commit: 70281247d913b19523c4d0b27b4631004a3a38fd
Parents: b0b8fcb
Author: Devaraj K <de...@apache.org>
Authored: Wed Jun 17 13:54:09 2015 +0530
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:00 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                   | 3 +++
 .../apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java  | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/70281247/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3679bf8..b1659a4 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -630,6 +630,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3714. AM proxy filter can not get RM webapp address from
     yarn.resourcemanager.hostname.rm-id. (Masatake Iwasaki via xgong)
 
+    YARN-3617. Fix WindowsResourceCalculatorPlugin.getCpuFrequency()
+    returning always -1. (J.Andreina via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/70281247/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
index 0e89118..cdbf525 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/WindowsResourceCalculatorPlugin.java
@@ -157,7 +157,7 @@ public class WindowsResourceCalculatorPlugin extends ResourceCalculatorPlugin {
   @Override
   public long getCpuFrequency() {
     refreshIfNeeded();
-    return -1;
+    return cpuFrequencyKhz;
   }
 
   /** {@inheritDoc} */


[19/50] [abbrv] hadoop git commit: HDFS-8576. Lease recovery should return true if the lease can be released and the file can be closed. Contributed by J.Andreina

Posted by zj...@apache.org.
HDFS-8576.  Lease recovery should return true if the lease can be released and the file can be closed.  Contributed by J.Andreina


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9e1a876b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9e1a876b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9e1a876b

Branch: refs/heads/YARN-2928
Commit: 9e1a876b1305f374379ab64d6cc3adffb30b4549
Parents: 0eb0d6e
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 15 16:07:38 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:06 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../hdfs/server/namenode/FSNamesystem.java      | 22 ++++++----
 .../apache/hadoop/hdfs/TestLeaseRecovery.java   | 46 ++++++++++++++++++++
 3 files changed, 62 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e1a876b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c98d918..21acf98 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1011,6 +1011,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8595. TestCommitBlockSynchronization fails in branch-2.7. (Patch
     applies to all branches). (Arpit Agarwal)
 
+    HDFS-8576.  Lease recovery should return true if the lease can be released
+    and the file can be closed.  (J.Andreina via szetszwo)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e1a876b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index f962373..518adb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -2616,7 +2616,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param src the path of the file to start lease recovery
    * @param holder the lease holder's name
    * @param clientMachine the client machine's name
-   * @return true if the file is already closed
+   * @return true if the file is already closed or
+   *         if the lease can be released and the file can be closed.
    * @throws IOException
    */
   boolean recoverLease(String src, String holder, String clientMachine)
@@ -2643,7 +2644,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         dir.checkPathAccess(pc, iip, FsAction.WRITE);
       }
   
-      recoverLeaseInternal(RecoverLeaseOp.RECOVER_LEASE,
+      return recoverLeaseInternal(RecoverLeaseOp.RECOVER_LEASE,
           iip, src, holder, clientMachine, true);
     } catch (StandbyException se) {
       skipSync = true;
@@ -2656,7 +2657,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         getEditLog().logSync();
       }
     }
-    return false;
   }
 
   enum RecoverLeaseOp {
@@ -2672,12 +2672,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
-  void recoverLeaseInternal(RecoverLeaseOp op, INodesInPath iip,
+  boolean recoverLeaseInternal(RecoverLeaseOp op, INodesInPath iip,
       String src, String holder, String clientMachine, boolean force)
       throws IOException {
     assert hasWriteLock();
     INodeFile file = iip.getLastINode().asFile();
-    if (file != null && file.isUnderConstruction()) {
+    if (file.isUnderConstruction()) {
       //
       // If the file is under construction , then it must be in our
       // leases. Find the appropriate lease record.
@@ -2710,7 +2710,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         // close only the file src
         LOG.info("recoverLease: " + lease + ", src=" + src +
           " from client " + clientName);
-        internalReleaseLease(lease, src, iip, holder);
+        return internalReleaseLease(lease, src, iip, holder);
       } else {
         assert lease.getHolder().equals(clientName) :
           "Current lease holder " + lease.getHolder() +
@@ -2722,11 +2722,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         if (lease.expiredSoftLimit()) {
           LOG.info("startFile: recover " + lease + ", src=" + src + " client "
               + clientName);
-          boolean isClosed = internalReleaseLease(lease, src, iip, null);
-          if(!isClosed)
+          if (internalReleaseLease(lease, src, iip, null)) {
+            return true;
+          } else {
             throw new RecoveryInProgressException(
                 op.getExceptionMessage(src, holder, clientMachine,
                     "lease recovery is in progress. Try again later."));
+          }
         } else {
           final BlockInfo lastBlock = file.getLastBlock();
           if (lastBlock != null
@@ -2743,7 +2745,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           }
         }
       }
-    }
+    } else {
+      return true;
+     }
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e1a876b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
index 15580a5..c9f3842 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
@@ -43,6 +44,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.TestInterDatanodePr
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.junit.After;
 import org.junit.Test;
@@ -212,4 +214,48 @@ public class TestLeaseRecovery {
     assertTrue("File should be closed", newdfs.recoverLease(file));
 
   }
+
+  /**
+   * Recover the lease on a file and append file from another client.
+   */
+  @Test
+  public void testLeaseRecoveryAndAppend() throws Exception {
+    Configuration conf = new Configuration();
+    try{
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    Path file = new Path("/testLeaseRecovery");
+    DistributedFileSystem dfs = cluster.getFileSystem();
+
+    // create a file with 0 bytes
+    FSDataOutputStream out = dfs.create(file);
+    out.hflush();
+    out.hsync();
+
+    // abort the original stream
+    ((DFSOutputStream) out.getWrappedStream()).abort();
+    DistributedFileSystem newdfs =
+        (DistributedFileSystem) FileSystem.newInstance
+        (cluster.getConfiguration(0));
+
+    // Append to a file , whose lease is held by another client should fail
+    try {
+        newdfs.append(file);
+        fail("Append to a file(lease is held by another client) should fail");
+    } catch (RemoteException e) {
+      assertTrue(e.getMessage().contains("file lease is currently owned"));
+    }
+
+    // Lease recovery on first try should be successful
+    boolean recoverLease = newdfs.recoverLease(file);
+    assertTrue(recoverLease);
+    FSDataOutputStream append = newdfs.append(file);
+    append.write("test".getBytes());
+    append.close();
+    }finally{
+      if (cluster != null) {
+        cluster.shutdown();
+        cluster = null;
+      }
+    }
+  }
 }


[25/50] [abbrv] hadoop git commit: HDFS-8592. SafeModeException never get unwrapped. Contributed by Haohui Mai.

Posted by zj...@apache.org.
HDFS-8592. SafeModeException never get unwrapped. Contributed by Haohui Mai.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/388f1ff9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/388f1ff9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/388f1ff9

Branch: refs/heads/YARN-2928
Commit: 388f1ff9b79e3e15fd2f29c817c29fc78517e72a
Parents: 5032eb9
Author: Haohui Mai <wh...@apache.org>
Authored: Mon Jun 15 22:07:24 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:07 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt              |  2 ++
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java | 11 ++++++++---
 .../hadoop/hdfs/server/namenode/SafeModeException.java   |  5 ++---
 .../test/java/org/apache/hadoop/hdfs/TestSafeMode.java   |  4 ++--
 4 files changed, 14 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/388f1ff9/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 79e7820..0ae2882 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -895,6 +895,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8607. TestFileCorruption doesn't work as expected. (Walter Su via
     Arpit Agarwal)
 
+    HDFS-8592. SafeModeException never get unwrapped. (wheat9)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/388f1ff9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 518adb4..40c71ea 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1323,7 +1323,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void checkNameNodeSafeMode(String errorMsg)
       throws RetriableException, SafeModeException {
     if (isInSafeMode()) {
-      SafeModeException se = new SafeModeException(errorMsg, safeMode);
+      SafeModeException se = newSafemodeException(errorMsg);
       if (haEnabled && haContext != null
           && haContext.getState().getServiceState() == HAServiceState.ACTIVE
           && shouldRetrySafeMode(this.safeMode)) {
@@ -1334,6 +1334,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
+  private SafeModeException newSafemodeException(String errorMsg) {
+    return new SafeModeException(errorMsg + ". Name node is in safe " +
+        "mode.\n" + safeMode.getTurnOffTip());
+  }
+
   boolean isPermissionEnabled() {
     return isPermissionEnabled;
   }
@@ -1803,8 +1808,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       for (LocatedBlock b : ret.blocks.getLocatedBlocks()) {
         // if safemode & no block locations yet then throw safemodeException
         if ((b.getLocations() == null) || (b.getLocations().length == 0)) {
-          SafeModeException se = new SafeModeException(
-              "Zero blocklocations for " + src, safeMode);
+          SafeModeException se = newSafemodeException(
+              "Zero blocklocations for " + src);
           if (haEnabled && haContext != null &&
               haContext.getState().getServiceState() == HAServiceState.ACTIVE) {
             throw new RetriableException(se);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/388f1ff9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
index 458f052..8080bcf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
@@ -32,8 +32,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 @InterfaceStability.Evolving
 public class SafeModeException extends IOException {
   private static final long serialVersionUID = 1L;
-
-  public SafeModeException(String text, FSNamesystem.SafeModeInfo mode ) {
-    super(text + ". Name node is in safe mode.\n" + mode.getTurnOffTip());
+  public SafeModeException(String msg) {
+    super(msg);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/388f1ff9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
index 80fe9ee..6cea7e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
@@ -295,8 +295,8 @@ public class TestSafeMode {
       fail(msg);
     } catch (RemoteException re) {
       assertEquals(SafeModeException.class.getName(), re.getClassName());
-      GenericTestUtils.assertExceptionContains(
-          "Name node is in safe mode", re);
+      GenericTestUtils.assertExceptionContains("Name node is in safe mode", re);
+    } catch (SafeModeException ignored) {
     } catch (IOException ioe) {
       fail(msg + " " + StringUtils.stringifyException(ioe));
     }


[09/50] [abbrv] hadoop git commit: HADOOP-12001. Moving CHANGES.txt up into 2.8.

Posted by zj...@apache.org.
HADOOP-12001. Moving CHANGES.txt up into 2.8.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/77bbe951
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/77bbe951
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/77bbe951

Branch: refs/heads/YARN-2928
Commit: 77bbe95131f96b145da6e7839cfe8b725b19cf2c
Parents: 68cc034
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Mon Jun 15 14:29:07 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:05 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/77bbe951/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index cdd396f..11c2f2a 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -855,6 +855,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-12074. in Shell.java#runCommand() rethrow InterruptedException as
     InterruptedIOException (Lavkesh Lahngir via vinayakumarb)
 
+    HADOOP-12001. Fixed LdapGroupsMapping to include configurable Posix UID and
+    GID attributes during the search. (Patrick White via vinodkv)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES
@@ -899,9 +902,6 @@ Release 2.7.1 - UNRELEASED
     HADOOP-12078. The default retry policy does not handle RetriableException
     correctly. (Arpit Agarwal)
 
-    HADOOP-12001. Fixed LdapGroupsMapping to include configurable Posix UID and
-    GID attributes during the search. (Patrick White via vinodkv)
-
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES


[23/50] [abbrv] hadoop git commit: YARN-3789. Improve logs for LeafQueue#activateApplications(). Contributed by Bibin A Chundatt.

Posted by zj...@apache.org.
YARN-3789. Improve logs for LeafQueue#activateApplications(). Contributed
by Bibin A Chundatt.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f28dc4f0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f28dc4f0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f28dc4f0

Branch: refs/heads/YARN-2928
Commit: f28dc4f030f6f25f4a8525be92cca632c4b4e860
Parents: 388f1ff
Author: Devaraj K <de...@apache.org>
Authored: Tue Jun 16 14:03:22 2015 +0530
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:07 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                    |  3 +++
 .../scheduler/capacity/LeafQueue.java              | 17 +++++++++--------
 2 files changed, 12 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f28dc4f0/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 99f2c64..92060ae 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -406,6 +406,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3785. Support for Resource as an argument during submitApp call in MockRM
     test class. (Sunil G via xgong)
 
+    YARN-3789. Improve logs for LeafQueue#activateApplications().
+    (Bibin A Chundatt via devaraj)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f28dc4f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
index 15d3289..8e39133 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/LeafQueue.java
@@ -56,7 +56,6 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
 import org.apache.hadoop.yarn.security.AccessType;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
@@ -604,7 +603,7 @@ public class LeafQueue extends AbstractCSQueue {
     for (Iterator<FiCaSchedulerApp> i=pendingApplications.iterator(); 
          i.hasNext(); ) {
       FiCaSchedulerApp application = i.next();
-      
+      ApplicationId applicationId = application.getApplicationId();
       // Check am resource limit
       Resource amIfStarted = 
         Resources.add(application.getAMResource(), queueUsage.getAMUsed());
@@ -624,7 +623,9 @@ public class LeafQueue extends AbstractCSQueue {
             " single application in queue, it is likely set too low." +
             " skipping enforcement to allow at least one application to start"); 
         } else {
-          LOG.info("not starting application as amIfStarted exceeds amLimit");
+          LOG.info("Not activating application " + applicationId
+              + " as  amIfStarted: " + amIfStarted + " exceeds amLimit: "
+              + amLimit);
           continue;
         }
       }
@@ -645,8 +646,9 @@ public class LeafQueue extends AbstractCSQueue {
             " single application in queue for user, it is likely set too low." +
             " skipping enforcement to allow at least one application to start"); 
         } else {
-          LOG.info("not starting application as amIfStarted exceeds " +
-            "userAmLimit");
+          LOG.info("Not activating application " + applicationId
+              + " for user: " + user + " as userAmIfStarted: "
+              + userAmIfStarted + " exceeds userAmLimit: " + userAMLimit);
           continue;
         }
       }
@@ -657,9 +659,8 @@ public class LeafQueue extends AbstractCSQueue {
       metrics.incAMUsed(application.getUser(), application.getAMResource());
       metrics.setAMResouceLimitForUser(application.getUser(), userAMLimit);
       i.remove();
-      LOG.info("Application " + application.getApplicationId() +
-          " from user: " + application.getUser() + 
-          " activated in queue: " + getQueueName());
+      LOG.info("Application " + applicationId + " from user: "
+          + application.getUser() + " activated in queue: " + getQueueName());
     }
   }
   


[08/50] [abbrv] hadoop git commit: HDFS-8583. Document that NFS gateway does not work with rpcbind on SLES 11. (Arpit Agarwal)

Posted by zj...@apache.org.
HDFS-8583. Document that NFS gateway does not work with rpcbind on SLES 11. (Arpit Agarwal)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/78cf309f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/78cf309f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/78cf309f

Branch: refs/heads/YARN-2928
Commit: 78cf309f2a6ee0cbf462b53d4b324d0290a86e19
Parents: e1d39ce
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jun 11 15:08:15 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:04 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md               | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/78cf309f/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 38c8721..b388f69 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -987,6 +987,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8566. HDFS documentation about debug commands wrongly identifies them
     as "hdfs dfs" commands (Surendra Singh Lilhore via Colin P. McCabe)
 
+    HDFS-8583. Document that NFS gateway does not work with rpcbind
+    on SLES 11. (Arpit Agarwal)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/78cf309f/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
index 4bca599..da7aa6f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
@@ -198,7 +198,7 @@ It's strongly recommended for the users to update a few configuration properties
 Start and stop NFS gateway service
 ----------------------------------
 
-Three daemons are required to provide NFS service: rpcbind (or portmap), mountd and nfsd. The NFS gateway process has both nfsd and mountd. It shares the HDFS root "/" as the only export. It is recommended to use the portmap included in NFS gateway package. Even though NFS gateway works with portmap/rpcbind provide by most Linux distributions, the package included portmap is needed on some Linux systems such as REHL6.2 due to an [rpcbind bug](https://bugzilla.redhat.com/show_bug.cgi?id=731542). More detailed discussions can be found in [HDFS-4763](https://issues.apache.org/jira/browse/HDFS-4763).
+Three daemons are required to provide NFS service: rpcbind (or portmap), mountd and nfsd. The NFS gateway process has both nfsd and mountd. It shares the HDFS root "/" as the only export. It is recommended to use the portmap included in NFS gateway package. Even though NFS gateway works with portmap/rpcbind provide by most Linux distributions, the package included portmap is needed on some Linux systems such as RHEL 6.2 and SLES 11, the former due to an [rpcbind bug](https://bugzilla.redhat.com/show_bug.cgi?id=731542). More detailed discussions can be found in [HDFS-4763](https://issues.apache.org/jira/browse/HDFS-4763).
 
 1.  Stop nfsv3 and rpcbind/portmap services provided by the platform (commands can be different on various Unix platforms):
 


[36/50] [abbrv] hadoop git commit: YARN-3148. Allow CORS related headers to passthrough in WebAppProxyServlet. Contributed by Varun Saxena.

Posted by zj...@apache.org.
YARN-3148. Allow CORS related headers to passthrough in
WebAppProxyServlet. Contributed by Varun Saxena.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c7ed62f3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c7ed62f3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c7ed62f3

Branch: refs/heads/YARN-2928
Commit: c7ed62f38601517003e88619b55ef9d29b49c727
Parents: 7028124
Author: Devaraj K <de...@apache.org>
Authored: Wed Jun 17 15:56:18 2015 +0530
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:00 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../server/webproxy/WebAppProxyServlet.java     |  5 +-
 .../server/webproxy/TestWebAppProxyServlet.java | 57 ++++++++++++++++++++
 3 files changed, 64 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ed62f3/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b1659a4..ae9716c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -409,6 +409,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3789. Improve logs for LeafQueue#activateApplications().
     (Bibin A Chundatt via devaraj)
 
+    YARN-3148. Allow CORS related headers to passthrough in WebAppProxyServlet.
+    (Varun Saxena via devaraj)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ed62f3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
index b1e355d..33f36f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServlet.java
@@ -81,7 +81,10 @@ public class WebAppProxyServlet extends HttpServlet {
         "Accept-Encoding",
         "Accept-Language",
         "Accept-Charset",
-        "Content-Type"));
+        "Content-Type",
+        "Origin",
+        "Access-Control-Request-Method",
+        "Access-Control-Request-Headers"));
 
   public static final String PROXY_USER_COOKIE_NAME = "proxy-user";
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c7ed62f3/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
index 2a2ca2c..8e68c38 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestWebAppProxyServlet.java
@@ -31,6 +31,7 @@ import java.net.HttpCookie;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URL;
+import java.util.Enumeration;
 import java.util.List;
 import java.util.Map;
 
@@ -72,6 +73,10 @@ public class TestWebAppProxyServlet {
 
   private static Server server;
   private static int originalPort = 0;
+  private static int numberOfHeaders = 0;
+  private static final String UNKNOWN_HEADER = "Unknown-Header";
+  private static boolean hasUnknownHeader = false;
+
 
   /**
    * Simple http server. Server should send answer with status 200
@@ -88,6 +93,9 @@ public class TestWebAppProxyServlet {
     originalPort = server.getConnectors()[0].getLocalPort();
     LOG.info("Running embedded servlet container at: http://localhost:"
         + originalPort);
+    // This property needs to be set otherwise CORS Headers will be dropped
+    // by HttpUrlConnection
+    System.setProperty("sun.net.http.allowRestrictedHeaders", "true");
   }
 
   @SuppressWarnings("serial")
@@ -96,6 +104,18 @@ public class TestWebAppProxyServlet {
     @Override
     protected void doGet(HttpServletRequest req, HttpServletResponse resp)
         throws ServletException, IOException {
+      int numHeaders = 0;
+      hasUnknownHeader = false;
+      @SuppressWarnings("unchecked")
+      Enumeration<String> names = req.getHeaderNames();
+      while(names.hasMoreElements()) {
+        String headerName = names.nextElement();
+        if (headerName.equals(UNKNOWN_HEADER)) {
+          hasUnknownHeader = true;
+        }
+        ++numHeaders;
+      }
+      numberOfHeaders = numHeaders;
       resp.setStatus(HttpServletResponse.SC_OK);
     }
 
@@ -206,6 +226,43 @@ public class TestWebAppProxyServlet {
     }
   }
 
+  @Test(timeout=5000)
+  public void testWebAppProxyPassThroughHeaders() throws Exception {
+    Configuration configuration = new Configuration();
+    configuration.set(YarnConfiguration.PROXY_ADDRESS, "localhost:9091");
+    configuration.setInt("hadoop.http.max.threads", 5);
+    WebAppProxyServerForTest proxy = new WebAppProxyServerForTest();
+    proxy.init(configuration);
+    proxy.start();
+
+    int proxyPort = proxy.proxy.proxyServer.getConnectorAddress(0).getPort();
+
+    try {
+      URL url = new URL("http://localhost:" + proxyPort + "/proxy/application_00_1");
+      HttpURLConnection proxyConn = (HttpURLConnection) url.openConnection();
+      // set headers
+      proxyConn.addRequestProperty("Origin", "http://www.someurl.com");
+      proxyConn.addRequestProperty("Access-Control-Request-Method", "GET");
+      proxyConn.addRequestProperty(
+          "Access-Control-Request-Headers", "Authorization");
+      proxyConn.addRequestProperty(UNKNOWN_HEADER, "unknown");
+      // Verify if four headers mentioned above have been added
+      assertEquals(proxyConn.getRequestProperties().size(), 4);
+      proxyConn.connect();
+      assertEquals(HttpURLConnection.HTTP_OK, proxyConn.getResponseCode());
+      // Verify if number of headers received by end server is 8.
+      // Eight headers include Accept, Host, Connection, User-Agent, Cookie,
+      // Origin, Access-Control-Request-Method and
+      // Access-Control-Request-Headers. Pls note that Unknown-Header is dropped
+      // by proxy as it is not in the list of allowed headers.
+      assertEquals(numberOfHeaders, 8);
+      assertFalse(hasUnknownHeader);
+    } finally {
+      proxy.close();
+    }
+  }
+
+
   /**
    * Test main method of WebAppProxyServer
    */


[26/50] [abbrv] hadoop git commit: YARN-3714. AM proxy filter can not get RM webapp address from yarn.resourcemanager.hostname.rm-id. Contributed by Masatake Iwasaki

Posted by zj...@apache.org.
YARN-3714. AM proxy filter can not get RM webapp address from
yarn.resourcemanager.hostname.rm-id. Contributed by Masatake Iwasaki


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/76dc2f53
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/76dc2f53
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/76dc2f53

Branch: refs/heads/YARN-2928
Commit: 76dc2f535343a806db5cc86f2d270c7d00412bee
Parents: 816ab87
Author: Xuan <xg...@apache.org>
Authored: Tue Jun 16 14:05:09 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:18:58 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../org/apache/hadoop/yarn/util/RMHAUtils.java  | 33 ++++++++------
 .../amfilter/TestAmFilterInitializer.java       | 47 ++++++++++++++++++++
 3 files changed, 70 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/76dc2f53/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 92060ae..3679bf8 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -627,6 +627,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3794. TestRMEmbeddedElector fails because of ambiguous LOG reference.
     (Chengbing Liu via devaraj)
 
+    YARN-3714. AM proxy filter can not get RM webapp address from
+    yarn.resourcemanager.hostname.rm-id. (Masatake Iwasaki via xgong)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76dc2f53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RMHAUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RMHAUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RMHAUtils.java
index a7e1ce9..2e996e9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RMHAUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RMHAUtils.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceTarget;
 import org.apache.hadoop.yarn.client.RMHAServiceTarget;
+import org.apache.hadoop.yarn.conf.HAUtil;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 @Private
@@ -71,25 +72,31 @@ public class RMHAUtils {
 
   public static List<String> getRMHAWebappAddresses(
       final YarnConfiguration conf) {
+    String prefix;
+    String defaultPort;
+    if (YarnConfiguration.useHttps(conf)) {
+      prefix = YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS;
+      defaultPort = ":" + YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT;
+    } else {
+      prefix =YarnConfiguration.RM_WEBAPP_ADDRESS;
+      defaultPort = ":" + YarnConfiguration.DEFAULT_RM_WEBAPP_PORT;
+    }
     Collection<String> rmIds =
         conf.getStringCollection(YarnConfiguration.RM_HA_IDS);
     List<String> addrs = new ArrayList<String>();
-    if (YarnConfiguration.useHttps(conf)) {
-      for (String id : rmIds) {
-        String addr = conf.get(
-            YarnConfiguration.RM_WEBAPP_HTTPS_ADDRESS + "." + id);
-        if (addr != null) {
-          addrs.add(addr);
+    for (String id : rmIds) {
+      String addr = conf.get(HAUtil.addSuffix(prefix, id));
+      if (addr == null) {
+        String hostname =
+            conf.get(HAUtil.addSuffix(YarnConfiguration.RM_HOSTNAME, id));
+        if (hostname != null) {
+          addr = hostname + defaultPort;
         }
       }
-    } else {
-      for (String id : rmIds) {
-        String addr = conf.get(YarnConfiguration.RM_WEBAPP_ADDRESS + "." + id);
-        if (addr != null) {
-          addrs.add(addr);
-        }
+      if (addr != null) {
+        addrs.add(addr);
       }
     }
     return addrs;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/76dc2f53/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilterInitializer.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilterInitializer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilterInitializer.java
index bec62ce..63c2cf3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilterInitializer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilterInitializer.java
@@ -135,6 +135,15 @@ public class TestAmFilterInitializer extends TestCase {
     assertEquals(WebAppUtils.getResolvedRMWebAppURLWithoutScheme(conf),
         proxyHosts.get(0));
 
+    // Check conf in which only RM hostname is set
+    conf = new Configuration(false);
+    conf.set(YarnConfiguration.RM_WEBAPP_ADDRESS,
+        "${yarn.resourcemanager.hostname}:8088"); // default in yarn-default.xml
+    conf.set(YarnConfiguration.RM_HOSTNAME, "host1");
+    proxyHosts = WebAppUtils.getProxyHostsAndPortsForAmFilter(conf);
+    assertEquals(1, proxyHosts.size());
+    assertEquals("host1:8088", proxyHosts.get(0));
+
     // Check PROXY_ADDRESS has priority
     conf = new Configuration(false);
     conf.set(YarnConfiguration.PROXY_ADDRESS, "host1:1000");
@@ -188,6 +197,44 @@ public class TestAmFilterInitializer extends TestCase {
     Collections.sort(proxyHosts);
     assertEquals("host5:5000", proxyHosts.get(0));
     assertEquals("host6:6000", proxyHosts.get(1));
+
+    // Check config without explicit RM_WEBAPP_ADDRESS settings (RM HA)
+    conf = new Configuration(false);
+    conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    conf.set(YarnConfiguration.RM_HA_IDS, "rm1,rm2,rm3");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm1", "host2");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm2", "host3");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm3", "host4");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm4", "dummy");
+    proxyHosts = WebAppUtils.getProxyHostsAndPortsForAmFilter(conf);
+    assertEquals(3, proxyHosts.size());
+    Collections.sort(proxyHosts);
+    assertEquals("host2:" + YarnConfiguration.DEFAULT_RM_WEBAPP_PORT,
+        proxyHosts.get(0));
+    assertEquals("host3:" + YarnConfiguration.DEFAULT_RM_WEBAPP_PORT,
+        proxyHosts.get(1));
+    assertEquals("host4:" + YarnConfiguration.DEFAULT_RM_WEBAPP_PORT,
+        proxyHosts.get(2));
+
+    // Check config without explicit RM_WEBAPP_HTTPS_ADDRESS settings (RM HA)
+    conf = new Configuration(false);
+    conf.set(YarnConfiguration.YARN_HTTP_POLICY_KEY,
+        HttpConfig.Policy.HTTPS_ONLY.toString());
+    conf.setBoolean(YarnConfiguration.RM_HA_ENABLED, true);
+    conf.set(YarnConfiguration.RM_HA_IDS, "rm1,rm2,rm3");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm1", "host2");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm2", "host3");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm3", "host4");
+    conf.set(YarnConfiguration.RM_HOSTNAME + ".rm4", "dummy");
+    proxyHosts = WebAppUtils.getProxyHostsAndPortsForAmFilter(conf);
+    assertEquals(3, proxyHosts.size());
+    Collections.sort(proxyHosts);
+    assertEquals("host2:" + YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT,
+        proxyHosts.get(0));
+    assertEquals("host3:" + YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT,
+        proxyHosts.get(1));
+    assertEquals("host4:" + YarnConfiguration.DEFAULT_RM_WEBAPP_HTTPS_PORT,
+        proxyHosts.get(2));
   }
 
   class MockAmFilterInitializer extends AmFilterInitializer {


[47/50] [abbrv] hadoop git commit: HADOOP-11965. determine-flaky-tests needs a summary mode. Contributed by Yufei Gu,

Posted by zj...@apache.org.
HADOOP-11965. determine-flaky-tests needs a summary mode. Contributed by Yufei Gu,


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8119664b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8119664b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8119664b

Branch: refs/heads/YARN-2928
Commit: 8119664bf5cc9cd5245561af7c081d88e23d89e6
Parents: 19551cf
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Wed Jun 17 15:48:29 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:02 2015 -0700

----------------------------------------------------------------------
 dev-support/determine-flaky-tests-hadoop.py     | 57 +++++++++++++++++---
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 2 files changed, 52 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8119664b/dev-support/determine-flaky-tests-hadoop.py
----------------------------------------------------------------------
diff --git a/dev-support/determine-flaky-tests-hadoop.py b/dev-support/determine-flaky-tests-hadoop.py
index ce152ba..8644299 100755
--- a/dev-support/determine-flaky-tests-hadoop.py
+++ b/dev-support/determine-flaky-tests-hadoop.py
@@ -62,12 +62,19 @@ import time
 DEFAULT_JENKINS_URL = "https://builds.apache.org"
 DEFAULT_JOB_NAME = "Hadoop-Common-trunk"
 DEFAULT_NUM_PREVIOUS_DAYS = 14
+DEFAULT_TOP_NUM_FAILED_TEST = -1
 
 SECONDS_PER_DAY = 86400
 
 # total number of runs to examine
 numRunsToExamine = 0
 
+#summary mode
+summary_mode = False
+
+#total number of errors
+error_count = 0
+
 """ Parse arguments """
 def parse_args():
   parser = OptionParser()
@@ -80,6 +87,10 @@ def parse_args():
   parser.add_option("-n", "--num-days", type="int",
                     dest="num_prev_days", help="Number of days to examine",
                     default=DEFAULT_NUM_PREVIOUS_DAYS)
+  parser.add_option("-t", "--top", type="int",
+                    dest="num_failed_tests",
+                    help="Summary Mode, only show top number of failed tests",
+                    default=DEFAULT_TOP_NUM_FAILED_TEST)
 
   (options, args) = parser.parse_args()
   if args:
@@ -100,6 +111,7 @@ def load_url_data(url):
  
 """ List all builds of the target project. """
 def list_builds(jenkins_url, job_name):
+  global summary_mode
   url = "%(jenkins)s/job/%(job_name)s/api/json?tree=builds[url,result,timestamp]" % dict(
       jenkins=jenkins_url,
       job_name=job_name)
@@ -108,19 +120,25 @@ def list_builds(jenkins_url, job_name):
     data = load_url_data(url)
 
   except:
-    logging.error("Could not fetch: %s" % url)
+    if not summary_mode:
+      logging.error("Could not fetch: %s" % url)
+    error_count += 1
     raise
   return data['builds']
 
 """ Find the names of any tests which failed in the given build output URL. """
 def find_failing_tests(testReportApiJson, jobConsoleOutput):
+  global summary_mode
+  global error_count
   ret = set()
   try:
     data = load_url_data(testReportApiJson)
 
   except:
-    logging.error("    Could not open testReport, check " +
+    if not summary_mode:
+      logging.error("    Could not open testReport, check " +
         jobConsoleOutput + " for why it was reported failed")
+    error_count += 1
     return ret
 
   for suite in data['suites']:
@@ -130,7 +148,7 @@ def find_failing_tests(testReportApiJson, jobConsoleOutput):
       if (status == 'REGRESSION' or status == 'FAILED' or (errDetails is not None)):
         ret.add(cs['className'] + "." + cs['name'])
 
-  if len(ret) == 0:
+  if len(ret) == 0 and (not summary_mode):
     logging.info("    No failed tests in testReport, check " +
         jobConsoleOutput + " for why it was reported failed.")
   return ret
@@ -138,6 +156,7 @@ def find_failing_tests(testReportApiJson, jobConsoleOutput):
 """ Iterate runs of specfied job within num_prev_days and collect results """
 def find_flaky_tests(jenkins_url, job_name, num_prev_days):
   global numRunsToExamine
+  global summary_mode
   all_failing = dict()
   # First list all builds
   builds = list_builds(jenkins_url, job_name)
@@ -153,7 +172,8 @@ def find_flaky_tests(jenkins_url, job_name, num_prev_days):
   tnum = len(builds)
   num = len(failing_build_urls)
   numRunsToExamine = tnum
-  logging.info("    THERE ARE " + str(num) + " builds (out of " + str(tnum)
+  if not summary_mode:
+    logging.info("    THERE ARE " + str(num) + " builds (out of " + str(tnum)
       + ") that have failed tests in the past " + str(num_prev_days) + " days"
       + ((".", ", as listed below:\n")[num > 0]))
 
@@ -165,17 +185,20 @@ def find_flaky_tests(jenkins_url, job_name, num_prev_days):
 
     ts = float(failed_build_with_time[1]) / 1000.
     st = datetime.datetime.fromtimestamp(ts).strftime('%Y-%m-%d %H:%M:%S')
-    logging.info("===>%s" % str(testReport) + " (" + st + ")")
+    if not summary_mode:
+      logging.info("===>%s" % str(testReport) + " (" + st + ")")
     failing = find_failing_tests(testReportApiJson, jobConsoleOutput)
     if failing:
       for ftest in failing:
-        logging.info("    Failed test: %s" % ftest)
+        if not summary_mode:
+          logging.info("    Failed test: %s" % ftest)
         all_failing[ftest] = all_failing.get(ftest,0)+1
 
   return all_failing
 
 def main():
   global numRunsToExamine
+  global summary_mode
   logging.basicConfig(format='%(levelname)s:%(message)s', level=logging.INFO)
 
   # set up logger to write to stdout
@@ -189,16 +212,34 @@ def main():
   logging.info("****Recently FAILED builds in url: " + opts.jenkins_url
       + "/job/" + opts.job_name + "")
 
+  if opts.num_failed_tests != -1:
+    summary_mode = True
+
   all_failing = find_flaky_tests(opts.jenkins_url, opts.job_name,
       opts.num_prev_days)
   if len(all_failing) == 0:
     raise SystemExit(0)
-  logging.info("\nAmong " + str(numRunsToExamine) + " runs examined, all failed "
-      + "tests <#failedRuns: testName>:")
+
+  if summary_mode and opts.num_failed_tests < len(all_failing):
+    logging.info("\nAmong " + str(numRunsToExamine) +
+                 " runs examined, top " + str(opts.num_failed_tests) +
+                 " failed tests <#failedRuns: testName>:")
+  else:
+      logging.info("\nAmong " + str(numRunsToExamine) +
+                   " runs examined, all failed tests <#failedRuns: testName>:")
 
   # print summary section: all failed tests sorted by how many times they failed
+  line_count = 0
   for tn in sorted(all_failing, key=all_failing.get, reverse=True):
     logging.info("    " + str(all_failing[tn])+ ": " + tn)
+    if summary_mode:
+      line_count += 1
+      if line_count == opts.num_failed_tests:
+        break
+
+  if summary_mode and error_count > 0:
+    logging.info("\n" + str(error_count) + " errors found, you may "
+                 + "re-run in non summary mode to see error details.");
 
 if __name__ == "__main__":
   main()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8119664b/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 67e9c76..4b1b382 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -648,6 +648,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11971. Move test utilities for tracing from hadoop-hdfs to
     hadoop-common. (Masatake Iwasaki via aajisaka)
 
+    HADOOP-11965. determine-flaky-tests needs a summary mode.
+    (Yufei Gu via Yongjun Zhang)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp


[16/50] [abbrv] hadoop git commit: HDFS-8540. Mover should exit with NO_MOVE_BLOCK if no block can be moved. Contributed by surendra singh lilhore

Posted by zj...@apache.org.
HDFS-8540.  Mover should exit with NO_MOVE_BLOCK if no block can be moved.  Contributed by surendra singh lilhore


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fe8250df
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fe8250df
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fe8250df

Branch: refs/heads/YARN-2928
Commit: fe8250dff8895455ccf19ec07a84d55b0efa8c7f
Parents: 9e1a876
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Mon Jun 15 16:26:53 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:06 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../apache/hadoop/hdfs/server/mover/Mover.java  | 95 ++++++++++++++------
 .../hadoop/hdfs/server/mover/TestMover.java     | 29 ++++++
 .../hdfs/server/mover/TestStorageMover.java     | 18 ++--
 4 files changed, 107 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe8250df/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 21acf98..584d94d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -917,6 +917,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8521. Add VisibleForTesting annotation to
     BlockPoolSlice#selectReplicaToDelete. (cmccabe)
 
+    HDFS-8540.  Mover should exit with NO_MOVE_BLOCK if no block can be moved.
+    (surendra singh lilhore via szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe8250df/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
index 8715ce4..344b9fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
@@ -27,7 +27,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.BlockStoragePolicySpi;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -163,8 +162,7 @@ public class Mover {
   private ExitStatus run() {
     try {
       init();
-      boolean hasRemaining = new Processor().processNamespace();
-      return hasRemaining ? ExitStatus.IN_PROGRESS : ExitStatus.SUCCESS;
+      return new Processor().processNamespace().getExitStatus();
     } catch (IllegalArgumentException e) {
       System.out.println(e + ".  Exiting ...");
       return ExitStatus.ILLEGAL_ARGUMENTS;
@@ -262,11 +260,11 @@ public class Mover {
      * @return whether there is still remaining migration work for the next
      *         round
      */
-    private boolean processNamespace() throws IOException {
+    private Result processNamespace() throws IOException {
       getSnapshottableDirs();
-      boolean hasRemaining = false;
+      Result result = new Result();
       for (Path target : targetPaths) {
-        hasRemaining |= processPath(target.toUri().getPath());
+        processPath(target.toUri().getPath(), result);
       }
       // wait for pending move to finish and retry the failed migration
       boolean hasFailed = Dispatcher.waitForMoveCompletion(storages.targets
@@ -282,16 +280,15 @@ public class Mover {
         // Reset retry count if no failure.
         retryCount.set(0);
       }
-      hasRemaining |= hasFailed;
-      return hasRemaining;
+      result.updateHasRemaining(hasFailed);
+      return result;
     }
 
     /**
      * @return whether there is still remaing migration work for the next
      *         round
      */
-    private boolean processPath(String fullPath) {
-      boolean hasRemaining = false;
+    private void processPath(String fullPath, Result result) {
       for (byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME;;) {
         final DirectoryListing children;
         try {
@@ -299,73 +296,71 @@ public class Mover {
         } catch(IOException e) {
           LOG.warn("Failed to list directory " + fullPath
               + ". Ignore the directory and continue.", e);
-          return hasRemaining;
+          return;
         }
         if (children == null) {
-          return hasRemaining;
+          return;
         }
         for (HdfsFileStatus child : children.getPartialListing()) {
-          hasRemaining |= processRecursively(fullPath, child);
+          processRecursively(fullPath, child, result);
         }
         if (children.hasMore()) {
           lastReturnedName = children.getLastName();
         } else {
-          return hasRemaining;
+          return;
         }
       }
     }
 
     /** @return whether the migration requires next round */
-    private boolean processRecursively(String parent, HdfsFileStatus status) {
+    private void processRecursively(String parent, HdfsFileStatus status,
+        Result result) {
       String fullPath = status.getFullName(parent);
-      boolean hasRemaining = false;
       if (status.isDir()) {
         if (!fullPath.endsWith(Path.SEPARATOR)) {
           fullPath = fullPath + Path.SEPARATOR;
         }
 
-        hasRemaining = processPath(fullPath);
+        processPath(fullPath, result);
         // process snapshots if this is a snapshottable directory
         if (snapshottableDirs.contains(fullPath)) {
           final String dirSnapshot = fullPath + HdfsConstants.DOT_SNAPSHOT_DIR;
-          hasRemaining |= processPath(dirSnapshot);
+          processPath(dirSnapshot, result);
         }
       } else if (!status.isSymlink()) { // file
         try {
           if (!isSnapshotPathInCurrent(fullPath)) {
             // the full path is a snapshot path but it is also included in the
             // current directory tree, thus ignore it.
-            hasRemaining = processFile(fullPath, (HdfsLocatedFileStatus)status);
+            processFile(fullPath, (HdfsLocatedFileStatus) status, result);
           }
         } catch (IOException e) {
           LOG.warn("Failed to check the status of " + parent
               + ". Ignore it and continue.", e);
-          return false;
         }
       }
-      return hasRemaining;
     }
 
     /** @return true if it is necessary to run another round of migration */
-    private boolean processFile(String fullPath, HdfsLocatedFileStatus status) {
+    private void processFile(String fullPath, HdfsLocatedFileStatus status,
+        Result result) {
       final byte policyId = status.getStoragePolicy();
       // currently we ignore files with unspecified storage policy
       if (policyId == HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED) {
-        return false;
+        return;
       }
       final BlockStoragePolicy policy = blockStoragePolicies[policyId];
       if (policy == null) {
         LOG.warn("Failed to get the storage policy of file " + fullPath);
-        return false;
+        return;
       }
       final List<StorageType> types = policy.chooseStorageTypes(
           status.getReplication());
 
       final LocatedBlocks locatedBlocks = status.getBlockLocations();
-      boolean hasRemaining = false;
       final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete();
       List<LocatedBlock> lbs = locatedBlocks.getLocatedBlocks();
-      for(int i = 0; i < lbs.size(); i++) {
+      for (int i = 0; i < lbs.size(); i++) {
         if (i == lbs.size() - 1 && !lastBlkComplete) {
           // last block is incomplete, skip it
           continue;
@@ -375,12 +370,15 @@ public class Mover {
             lb.getStorageTypes());
         if (!diff.removeOverlap(true)) {
           if (scheduleMoves4Block(diff, lb)) {
-            hasRemaining |= (diff.existing.size() > 1 &&
-                diff.expected.size() > 1);
+            result.updateHasRemaining(diff.existing.size() > 1
+                && diff.expected.size() > 1);
+            // One block scheduled successfully, set noBlockMoved to false
+            result.setNoBlockMoved(false);
+          } else {
+            result.updateHasRemaining(true);
           }
         }
       }
-      return hasRemaining;
     }
 
     boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb) {
@@ -711,6 +709,45 @@ public class Mover {
     }
   }
 
+  private static class Result {
+
+    private boolean hasRemaining;
+    private boolean noBlockMoved;
+
+    Result() {
+      hasRemaining = false;
+      noBlockMoved = true;
+    }
+
+    boolean isHasRemaining() {
+      return hasRemaining;
+    }
+
+    boolean isNoBlockMoved() {
+      return noBlockMoved;
+    }
+
+    void updateHasRemaining(boolean hasRemaining) {
+      this.hasRemaining |= hasRemaining;
+    }
+
+    void setNoBlockMoved(boolean noBlockMoved) {
+      this.noBlockMoved = noBlockMoved;
+    }
+
+    /**
+     * @return SUCCESS if all moves are success and there is no remaining move.
+     *         Return NO_MOVE_BLOCK if there moves available but all the moves
+     *         cannot be scheduled. Otherwise, return IN_PROGRESS since there
+     *         must be some remaining moves.
+     */
+    ExitStatus getExitStatus() {
+      return !isHasRemaining() ? ExitStatus.SUCCESS
+          : isNoBlockMoved() ? ExitStatus.NO_MOVE_BLOCK
+              : ExitStatus.IN_PROGRESS;
+    }
+
+  }
   /**
    * Run a Mover in command line.
    *

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe8250df/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java
----------------------------------------------------------------------
diff --git 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
index f4bedab..49e2b23 100644
--- 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
@@ -328,6 +328,35 @@ public class TestMover {
     }
   }
 
+  @Test(timeout = 300000)
+  public void testMoveWhenStoragePolicyNotSatisfying() throws Exception {
+    // HDFS-8147
+    final Configuration conf = new HdfsConfiguration();
+    final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
+        .numDataNodes(3)
+        .storageTypes(
+            new StorageType[][] { { StorageType.DISK }, { StorageType.DISK },
+                { StorageType.DISK } }).build();
+    try {
+      cluster.waitActive();
+      final DistributedFileSystem dfs = cluster.getFileSystem();
+      final String file = "/testMoveWhenStoragePolicyNotSatisfying";
+      // write to DISK
+      final FSDataOutputStream out = dfs.create(new Path(file));
+      out.writeChars("testMoveWhenStoragePolicyNotSatisfying");
+      out.close();
+
+      // move to ARCHIVE
+      dfs.setStoragePolicy(new Path(file), "COLD");
+      int rc = ToolRunner.run(conf, new Mover.Cli(),
+          new String[] { "-p", file.toString() });
+      int exitcode = ExitStatus.NO_MOVE_BLOCK.getExitCode();
+      Assert.assertEquals("Exit code should be " + exitcode, exitcode, rc);
+    } finally {
+      cluster.shutdown();
+    }
+  }
+
   @Test
   public void testMoverFailedRetry() throws Exception {
     // HDFS-8147

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe8250df/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
index d8b40d4..3095f30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
@@ -219,7 +219,7 @@ public class TestStorageMover {
         verify(true);
 
         setStoragePolicy();
-        migrate();
+        migrate(ExitStatus.SUCCESS);
         verify(true);
       } finally {
         if (shutdown) {
@@ -250,8 +250,8 @@ public class TestStorageMover {
     /**
      * Run the migration tool.
      */
-    void migrate() throws Exception {
-      runMover();
+    void migrate(ExitStatus expectedExitCode) throws Exception {
+      runMover(expectedExitCode);
       Thread.sleep(5000); // let the NN finish deletion
     }
 
@@ -267,14 +267,14 @@ public class TestStorageMover {
       }
     }
 
-    private void runMover() throws Exception {
+    private void runMover(ExitStatus expectedExitCode) throws Exception {
       Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
       Map<URI, List<Path>> nnMap = Maps.newHashMap();
       for (URI nn : namenodes) {
         nnMap.put(nn, null);
       }
       int result = Mover.run(nnMap, conf);
-      Assert.assertEquals(ExitStatus.SUCCESS.getExitCode(), result);
+      Assert.assertEquals(expectedExitCode.getExitCode(), result);
     }
 
     private void verifyNamespace() throws Exception {
@@ -555,7 +555,7 @@ public class TestStorageMover {
     try {
       banner("start data migration");
       test.setStoragePolicy(); // set /foo to COLD
-      test.migrate();
+      test.migrate(ExitStatus.SUCCESS);
 
       // make sure the under construction block has not been migrated
       LocatedBlocks lbs = test.dfs.getClient().getLocatedBlocks(
@@ -605,7 +605,7 @@ public class TestStorageMover {
     try {
       test.runBasicTest(false);
       pathPolicyMap.moveAround(test.dfs);
-      test.migrate();
+      test.migrate(ExitStatus.SUCCESS);
 
       test.verify(true);
     } finally {
@@ -695,7 +695,7 @@ public class TestStorageMover {
       //test move a hot file to warm
       final Path file1 = new Path(pathPolicyMap.hot, "file1");
       test.dfs.rename(file1, pathPolicyMap.warm);
-      test.migrate();
+      test.migrate(ExitStatus.NO_MOVE_BLOCK);
       test.verifyFile(new Path(pathPolicyMap.warm, "file1"), WARM.getId());
     } finally {
       test.shutdownCluster();
@@ -753,7 +753,7 @@ public class TestStorageMover {
       { //test move a cold file to warm
         final Path file1 = new Path(pathPolicyMap.cold, "file1");
         test.dfs.rename(file1, pathPolicyMap.warm);
-        test.migrate();
+        test.migrate(ExitStatus.SUCCESS);
         test.verify(true);
       }
     } finally {


[32/50] [abbrv] hadoop git commit: HDFS-8606. Cleanup DFSOutputStream by removing unwanted changes from HDFS-8386. Contributed by Rakesh R

Posted by zj...@apache.org.
HDFS-8606. Cleanup DFSOutputStream by removing unwanted changes from HDFS-8386. Contributed by Rakesh R


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e0e7365d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e0e7365d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e0e7365d

Branch: refs/heads/YARN-2928
Commit: e0e7365dcd65d4b74704c96546ee37a5bb6a4f07
Parents: 3146ab1
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Jun 16 18:08:29 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:18:59 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                 | 3 +++
 .../main/java/org/apache/hadoop/hdfs/DFSOutputStream.java   | 9 +--------
 2 files changed, 4 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0e7365d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 2d4c062..a36e047 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -635,6 +635,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8540.  Mover should exit with NO_MOVE_BLOCK if no block can be moved.
     (surendra singh lilhore via szetszwo)
 
+    HDFS-8606. Cleanup DFSOutputStream by removing unwanted changes
+    from HDFS-8386. (Rakesh R via szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e0e7365d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 695e6da..4622be6 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -899,16 +899,9 @@ public class DFSOutputStream extends FSOutputSummer
   }
 
   /**
-   * Set the data streamer object.
-   */
-  protected synchronized void setStreamer(DataStreamer streamer) {
-    this.streamer = streamer;
-  }
-
-  /**
    * Returns the data streamer object.
    */
-  protected synchronized DataStreamer getStreamer() {
+  protected DataStreamer getStreamer() {
     return streamer;
   }
 }


[39/50] [abbrv] hadoop git commit: HDFS-8238. Move ClientProtocol to the hdfs-client. Contributed by Takanobu Asanuma.

Posted by zj...@apache.org.
HDFS-8238. Move ClientProtocol to the hdfs-client. Contributed by Takanobu Asanuma.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/eb2ccade
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/eb2ccade
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/eb2ccade

Branch: refs/heads/YARN-2928
Commit: eb2ccade523c3f51268a22414ce9e260f0ae792a
Parents: 5a4ccbd
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Jun 17 16:16:36 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:00 2015 -0700

----------------------------------------------------------------------
 .../hdfs/client/HdfsClientConfigKeys.java       |    1 +
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 1459 ++++++++++++++++++
 .../hdfs/server/namenode/SafeModeException.java |   38 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |    3 +-
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 1459 ------------------
 .../hdfs/server/namenode/SafeModeException.java |   38 -
 6 files changed, 1500 insertions(+), 1498 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb2ccade/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
index 26283aa..e6d579b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java
@@ -39,6 +39,7 @@ public interface HdfsClientConfigKeys {
   String  DFS_NAMENODE_HTTPS_ADDRESS_KEY = "dfs.namenode.https-address";
   String DFS_HA_NAMENODES_KEY_PREFIX = "dfs.ha.namenodes";
   int DFS_NAMENODE_RPC_PORT_DEFAULT = 8020;
+  String DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY = "dfs.namenode.kerberos.principal";
 
   /** dfs.client.retry configuration properties */
   interface Retry {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb2ccade/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
new file mode 100644
index 0000000..7e27078
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -0,0 +1,1459 @@
+/**
+ * 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 java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.EnumSet;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Options.Rename;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.inotify.EventBatchList;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.AtMostOnce;
+import org.apache.hadoop.io.retry.Idempotent;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.KerberosInfo;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenInfo;
+
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
+
+/**********************************************************************
+ * ClientProtocol is used by user code via the DistributedFileSystem class to
+ * communicate with the NameNode.  User code can manipulate the directory
+ * namespace, as well as open/close file streams, etc.
+ *
+ **********************************************************************/
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+@KerberosInfo(
+    serverPrincipal = DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
+@TokenInfo(DelegationTokenSelector.class)
+public interface ClientProtocol {
+
+  /**
+   * Until version 69, this class ClientProtocol served as both
+   * the client interface to the NN AND the RPC protocol used to 
+   * communicate with the NN.
+   * 
+   * This class is used by both the DFSClient and the 
+   * NN server side to insulate from the protocol serialization.
+   * 
+   * If you are adding/changing this interface then you need to 
+   * change both this class and ALSO related protocol buffer
+   * wire protocol definition in ClientNamenodeProtocol.proto.
+   * 
+   * For more details on protocol buffer wire protocol, please see 
+   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
+   * 
+   * The log of historical changes can be retrieved from the svn).
+   * 69: Eliminate overloaded method names.
+   * 
+   * 69L is the last version id when this class was used for protocols
+   *  serialization. DO not update this version any further. 
+   */
+  public static final long versionID = 69L;
+  
+  ///////////////////////////////////////
+  // File contents
+  ///////////////////////////////////////
+  /**
+   * Get locations of the blocks of the specified file within the specified range.
+   * DataNode locations for each block are sorted by
+   * the proximity to the client.
+   * <p>
+   * Return {@link LocatedBlocks} which contains
+   * file length, blocks and their locations.
+   * DataNode locations for each block are sorted by
+   * the distance to the client's address.
+   * <p>
+   * The client will then have to contact 
+   * one of the indicated DataNodes to obtain the actual data.
+   * 
+   * @param src file name
+   * @param offset range start offset
+   * @param length range length
+   *
+   * @return file length and array of blocks with their locations
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> does not exist
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public LocatedBlocks getBlockLocations(String src,
+                                         long offset,
+                                         long length) 
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
+
+  /**
+   * Get server default values for a number of configuration params.
+   * @return a set of server default configuration values
+   * @throws IOException
+   */
+  @Idempotent
+  public FsServerDefaults getServerDefaults() throws IOException;
+
+  /**
+   * Create a new file entry in the namespace.
+   * <p>
+   * This will create an empty file specified by the source path.
+   * The path should reflect a full path originated at the root.
+   * The name-node does not have a notion of "current" directory for a client.
+   * <p>
+   * Once created, the file is visible and available for read to other clients.
+   * Although, other clients cannot {@link #delete(String, boolean)}, re-create or 
+   * {@link #rename(String, String)} it until the file is completed
+   * or explicitly as a result of lease expiration.
+   * <p>
+   * Blocks have a maximum size.  Clients that intend to create
+   * multi-block files must also use 
+   * {@link #addBlock}
+   *
+   * @param src path of the file being created.
+   * @param masked masked permission.
+   * @param clientName name of the current client.
+   * @param flag indicates whether the file should be 
+   * overwritten if it already exists or create if it does not exist or append.
+   * @param createParent create missing parent directory if true
+   * @param replication block replication factor.
+   * @param blockSize maximum block size.
+   * @param supportedVersions CryptoProtocolVersions supported by the client
+   * 
+   * @return the status of the created file, it could be null if the server
+   *           doesn't support returning the file status
+   * @throws AccessControlException If access is denied
+   * @throws AlreadyBeingCreatedException if the path does not exist.
+   * @throws DSQuotaExceededException If file creation violates disk space 
+   *           quota restriction
+   * @throws FileAlreadyExistsException If file <code>src</code> already exists
+   * @throws FileNotFoundException If parent of <code>src</code> does not exist
+   *           and <code>createParent</code> is false
+   * @throws ParentNotDirectoryException If parent of <code>src</code> is not a
+   *           directory.
+   * @throws NSQuotaExceededException If file creation violates name space 
+   *           quota restriction
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   *
+   * RuntimeExceptions:
+   * @throws InvalidPathException Path <code>src</code> is invalid
+   * <p>
+   * <em>Note that create with {@link CreateFlag#OVERWRITE} is idempotent.</em>
+   */
+  @AtMostOnce
+  public HdfsFileStatus create(String src, FsPermission masked,
+      String clientName, EnumSetWritable<CreateFlag> flag,
+      boolean createParent, short replication, long blockSize, 
+      CryptoProtocolVersion[] supportedVersions)
+      throws AccessControlException, AlreadyBeingCreatedException,
+      DSQuotaExceededException, FileAlreadyExistsException,
+      FileNotFoundException, NSQuotaExceededException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      SnapshotAccessControlException, IOException;
+
+  /**
+   * Append to the end of the file. 
+   * @param src path of the file being created.
+   * @param clientName name of the current client.
+   * @param flag indicates whether the data is appended to a new block.
+   * @return wrapper with information about the last partial block and file
+   *    status if any
+   * @throws AccessControlException if permission to append file is 
+   * denied by the system. As usually on the client side the exception will 
+   * be wrapped into {@link org.apache.hadoop.ipc.RemoteException}.
+   * Allows appending to an existing file if the server is
+   * configured with the parameter dfs.support.append set to true, otherwise
+   * throws an IOException.
+   * 
+   * @throws AccessControlException If permission to append to file is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws DSQuotaExceededException If append violates disk space quota 
+   *           restriction
+   * @throws SafeModeException append not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred.
+   *
+   * RuntimeExceptions:
+   * @throws UnsupportedOperationException if append is not supported
+   */
+  @AtMostOnce
+  public LastBlockWithStatus append(String src, String clientName,
+      EnumSetWritable<CreateFlag> flag) throws AccessControlException,
+      DSQuotaExceededException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Set replication for an existing file.
+   * <p>
+   * The NameNode sets replication to the new value and returns.
+   * The actual block replication is not expected to be performed during  
+   * this method call. The blocks will be populated or removed in the 
+   * background as the result of the routine block maintenance procedures.
+   * 
+   * @param src file name
+   * @param replication new replication
+   * 
+   * @return true if successful;
+   *         false if file does not exist or is a directory
+   *
+   * @throws AccessControlException If access is denied
+   * @throws DSQuotaExceededException If replication violates disk space 
+   *           quota restriction
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException not allowed in safemode
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public boolean setReplication(String src, short replication)
+      throws AccessControlException, DSQuotaExceededException,
+      FileNotFoundException, SafeModeException, UnresolvedLinkException,
+      SnapshotAccessControlException, IOException;
+
+  /**
+   * Get all the available block storage policies.
+   * @return All the in-use block storage policies currently.
+   */
+  @Idempotent
+  public BlockStoragePolicy[] getStoragePolicies() throws IOException;
+
+  /**
+   * Set the storage policy for a file/directory
+   * @param src Path of an existing file/directory. 
+   * @param policyName The name of the storage policy
+   * @throws SnapshotAccessControlException If access is denied
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
+   * @throws FileNotFoundException If file/dir <code>src</code> is not found
+   * @throws QuotaExceededException If changes violate the quota restriction
+   */
+  @Idempotent
+  public void setStoragePolicy(String src, String policyName)
+      throws SnapshotAccessControlException, UnresolvedLinkException,
+      FileNotFoundException, QuotaExceededException, IOException;
+
+  /**
+   * Set permissions for an existing file/directory.
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void setPermission(String src, FsPermission permission)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Set Owner of a path (i.e. a file or a directory).
+   * The parameters username and groupname cannot both be null.
+   * @param src file path
+   * @param username If it is null, the original username remains unchanged.
+   * @param groupname If it is null, the original groupname remains unchanged.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void setOwner(String src, String username, String groupname)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * The client can give up on a block by calling abandonBlock().
+   * The client can then either obtain a new block, or complete or abandon the 
+   * file.
+   * Any partial writes to the block will be discarded.
+   * 
+   * @param b         Block to abandon
+   * @param fileId    The id of the file where the block resides.  Older clients
+   *                    will pass GRANDFATHER_INODE_ID here.
+   * @param src       The path of the file where the block resides.
+   * @param holder    Lease holder.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void abandonBlock(ExtendedBlock b, long fileId,
+      String src, String holder)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
+
+  /**
+   * A client that wants to write an additional block to the 
+   * indicated filename (which must currently be open for writing)
+   * should call addBlock().  
+   *
+   * addBlock() allocates a new block and datanodes the block data
+   * should be replicated to.
+   * 
+   * addBlock() also commits the previous block by reporting
+   * to the name-node the actual generation stamp and the length
+   * of the block that the client has transmitted to data-nodes.
+   *
+   * @param src the file being created
+   * @param clientName the name of the client that adds the block
+   * @param previous  previous block
+   * @param excludeNodes a list of nodes that should not be
+   * allocated for the current block
+   * @param fileId the id uniquely identifying a file
+   * @param favoredNodes the list of nodes where the client wants the blocks.
+   *          Nodes are identified by either host name or address.
+   *
+   * @return LocatedBlock allocated block information.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws NotReplicatedYetException previous blocks of the file are not
+   *           replicated yet. Blocks cannot be added until replication
+   *           completes.
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public LocatedBlock addBlock(String src, String clientName,
+      ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId, 
+      String[] favoredNodes)
+      throws AccessControlException, FileNotFoundException,
+      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
+      IOException;
+
+  /** 
+   * Get a datanode for an existing pipeline.
+   * 
+   * @param src the file being written
+   * @param fileId the ID of the file being written
+   * @param blk the block being written
+   * @param existings the existing nodes in the pipeline
+   * @param excludes the excluded nodes
+   * @param numAdditionalNodes number of additional datanodes
+   * @param clientName the name of the client
+   * 
+   * @return the located block.
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public LocatedBlock getAdditionalDatanode(final String src,
+      final long fileId, final ExtendedBlock blk,
+      final DatanodeInfo[] existings,
+      final String[] existingStorageIDs,
+      final DatanodeInfo[] excludes,
+      final int numAdditionalNodes, final String clientName
+      ) throws AccessControlException, FileNotFoundException,
+          SafeModeException, UnresolvedLinkException, IOException;
+
+  /**
+   * The client is done writing data to the given filename, and would 
+   * like to complete it.  
+   *
+   * The function returns whether the file has been closed successfully.
+   * If the function returns false, the caller should try again.
+   * 
+   * close() also commits the last block of file by reporting
+   * to the name-node the actual generation stamp and the length
+   * of the block that the client has transmitted to data-nodes.
+   *
+   * A call to complete() will not return true until all the file's
+   * blocks have been replicated the minimum number of times.  Thus,
+   * DataNode failures may cause a client to call complete() several
+   * times before succeeding.
+   *
+   * @param src the file being created
+   * @param clientName the name of the client that adds the block
+   * @param last the last block info
+   * @param fileId the id uniquely identifying a file
+   *
+   * @return true if all file blocks are minimally replicated or false otherwise
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink 
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public boolean complete(String src, String clientName,
+                          ExtendedBlock last, long fileId)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, IOException;
+
+  /**
+   * The client wants to report corrupted blocks (blocks with specified
+   * locations on datanodes).
+   * @param blocks Array of located blocks to report
+   */
+  @Idempotent
+  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
+
+  ///////////////////////////////////////
+  // Namespace management
+  ///////////////////////////////////////
+  /**
+   * Rename an item in the file system namespace.
+   * @param src existing file or directory name.
+   * @param dst new name.
+   * @return true if successful, or false if the old name does not exist
+   * or if the new name already belongs to the namespace.
+   * 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException an I/O error occurred 
+   */
+  @AtMostOnce
+  public boolean rename(String src, String dst) 
+      throws UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Moves blocks from srcs to trg and delete srcs
+   * 
+   * @param trg existing file
+   * @param srcs - list of existing files (same block size, same replication)
+   * @throws IOException if some arguments are invalid
+   * @throws UnresolvedLinkException if <code>trg</code> or <code>srcs</code>
+   *           contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   */
+  @AtMostOnce
+  public void concat(String trg, String[] srcs) 
+      throws IOException, UnresolvedLinkException, SnapshotAccessControlException;
+
+  /**
+   * Rename src to dst.
+   * <ul>
+   * <li>Fails if src is a file and dst is a directory.
+   * <li>Fails if src is a directory and dst is a file.
+   * <li>Fails if the parent of dst does not exist or is a file.
+   * </ul>
+   * <p>
+   * Without OVERWRITE option, rename fails if the dst already exists.
+   * With OVERWRITE option, rename overwrites the dst, if it is a file 
+   * or an empty directory. Rename fails if dst is a non-empty directory.
+   * <p>
+   * This implementation of rename is atomic.
+   * <p>
+   * @param src existing file or directory name.
+   * @param dst new name.
+   * @param options Rename options
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws DSQuotaExceededException If rename violates disk space 
+   *           quota restriction
+   * @throws FileAlreadyExistsException If <code>dst</code> already exists and
+   *           <code>options</options> has {@link Rename#OVERWRITE} option
+   *           false.
+   * @throws FileNotFoundException If <code>src</code> does not exist
+   * @throws NSQuotaExceededException If rename violates namespace 
+   *           quota restriction
+   * @throws ParentNotDirectoryException If parent of <code>dst</code> 
+   *           is not a directory
+   * @throws SafeModeException rename not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> or
+   *           <code>dst</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @AtMostOnce
+  public void rename2(String src, String dst, Options.Rename... options)
+      throws AccessControlException, DSQuotaExceededException,
+      FileAlreadyExistsException, FileNotFoundException,
+      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Truncate file src to new size.
+   * <ul>
+   * <li>Fails if src is a directory.
+   * <li>Fails if src does not exist.
+   * <li>Fails if src is not closed.
+   * <li>Fails if new size is greater than current size.
+   * </ul>
+   * <p>
+   * This implementation of truncate is purely a namespace operation if truncate
+   * occurs at a block boundary. Requires DataNode block recovery otherwise.
+   * <p>
+   * @param src  existing file
+   * @param newLength  the target size
+   *
+   * @return true if client does not need to wait for block recovery,
+   * false if client needs to wait for block recovery.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException truncate not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public boolean truncate(String src, long newLength, String clientName)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Delete the given file or directory from the file system.
+   * <p>
+   * same as delete but provides a way to avoid accidentally 
+   * deleting non empty directories programmatically. 
+   * @param src existing name
+   * @param recursive if true deletes a non empty directory recursively,
+   * else throws an exception.
+   * @return true only if the existing file or directory was actually removed 
+   * from the file system.
+   * 
+   * @throws AccessControlException If access is denied
+   * @throws FileNotFoundException If file <code>src</code> is not found
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @AtMostOnce
+  public boolean delete(String src, boolean recursive)
+      throws AccessControlException, FileNotFoundException, SafeModeException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+  
+  /**
+   * Create a directory (or hierarchy of directories) with the given
+   * name and permission.
+   *
+   * @param src The path of the directory being created
+   * @param masked The masked permission of the directory being created
+   * @param createParent create missing parent directory if true
+   *
+   * @return True if the operation success.
+   *
+   * @throws AccessControlException If access is denied
+   * @throws FileAlreadyExistsException If <code>src</code> already exists
+   * @throws FileNotFoundException If parent of <code>src</code> does not exist
+   *           and <code>createParent</code> is false
+   * @throws NSQuotaExceededException If file creation violates quota restriction
+   * @throws ParentNotDirectoryException If parent of <code>src</code> 
+   *           is not a directory
+   * @throws SafeModeException create not allowed in safemode
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred.
+   *
+   * RunTimeExceptions:
+   * @throws InvalidPathException If <code>src</code> is invalid
+   */
+  @Idempotent
+  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
+      throws AccessControlException, FileAlreadyExistsException,
+      FileNotFoundException, NSQuotaExceededException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      SnapshotAccessControlException, IOException;
+
+  /**
+   * Get a partial listing of the indicated directory
+   *
+   * @param src the directory name
+   * @param startAfter the name to start listing after encoded in java UTF8
+   * @param needLocation if the FileStatus should contain block locations
+   *
+   * @return a partial listing starting after startAfter
+   *
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public DirectoryListing getListing(String src,
+                                     byte[] startAfter,
+                                     boolean needLocation)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
+  
+  /**
+   * Get listing of all the snapshottable directories
+   * 
+   * @return Information about all the current snapshottable directory
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
+      throws IOException;
+
+  ///////////////////////////////////////
+  // System issues and management
+  ///////////////////////////////////////
+
+  /**
+   * Client programs can cause stateful changes in the NameNode
+   * that affect other clients.  A client may obtain a file and 
+   * neither abandon nor complete it.  A client might hold a series
+   * of locks that prevent other clients from proceeding.
+   * Clearly, it would be bad if a client held a bunch of locks
+   * that it never gave up.  This can happen easily if the client
+   * dies unexpectedly.
+   * <p>
+   * So, the NameNode will revoke the locks and live file-creates
+   * for clients that it thinks have died.  A client tells the
+   * NameNode that it is still alive by periodically calling
+   * renewLease().  If a certain amount of time passes since
+   * the last call to renewLease(), the NameNode assumes the
+   * client has died.
+   *
+   * @throws AccessControlException permission denied
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void renewLease(String clientName) throws AccessControlException,
+      IOException;
+
+  /**
+   * Start lease recovery.
+   * Lightweight NameNode operation to trigger lease recovery
+   * 
+   * @param src path of the file to start lease recovery
+   * @param clientName name of the current client
+   * @return true if the file is already closed
+   * @throws IOException
+   */
+  @Idempotent
+  public boolean recoverLease(String src, String clientName) throws IOException;
+
+  public int GET_STATS_CAPACITY_IDX = 0;
+  public int GET_STATS_USED_IDX = 1;
+  public int GET_STATS_REMAINING_IDX = 2;
+  public int GET_STATS_UNDER_REPLICATED_IDX = 3;
+  public int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
+  public int GET_STATS_MISSING_BLOCKS_IDX = 5;
+  public int GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX = 6;
+  
+  /**
+   * Get a set of statistics about the filesystem.
+   * Right now, only seven values are returned.
+   * <ul>
+   * <li> [0] contains the total storage capacity of the system, in bytes.</li>
+   * <li> [1] contains the total used space of the system, in bytes.</li>
+   * <li> [2] contains the available storage of the system, in bytes.</li>
+   * <li> [3] contains number of under replicated blocks in the system.</li>
+   * <li> [4] contains number of blocks with a corrupt replica. </li>
+   * <li> [5] contains number of blocks without any good replicas left. </li>
+   * <li> [6] contains number of blocks which have replication factor
+   *          1 and have lost the only replica. </li>
+   * </ul>
+   * Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of 
+   * actual numbers to index into the array.
+   */
+  @Idempotent
+  public long[] getStats() throws IOException;
+
+  /**
+   * Get a report on the system's current datanodes.
+   * One DatanodeInfo object is returned for each DataNode.
+   * Return live datanodes if type is LIVE; dead datanodes if type is DEAD;
+   * otherwise all datanodes if type is ALL.
+   */
+  @Idempotent
+  public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
+      throws IOException;
+
+  /**
+   * Get a report on the current datanode storages.
+   */
+  @Idempotent
+  public DatanodeStorageReport[] getDatanodeStorageReport(
+      HdfsConstants.DatanodeReportType type) throws IOException;
+
+  /**
+   * Get the block size for the given file.
+   * @param filename The name of the file
+   * @return The number of bytes in each block
+   * @throws IOException
+   * @throws UnresolvedLinkException if the path contains a symlink. 
+   */
+  @Idempotent
+  public long getPreferredBlockSize(String filename) 
+      throws IOException, UnresolvedLinkException;
+
+  /**
+   * Enter, leave or get safe mode.
+   * <p>
+   * Safe mode is a name node state when it
+   * <ol><li>does not accept changes to name space (read-only), and</li>
+   * <li>does not replicate or delete blocks.</li></ol>
+   * 
+   * <p>
+   * Safe mode is entered automatically at name node startup.
+   * Safe mode can also be entered manually using
+   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}.
+   * <p>
+   * At startup the name node accepts data node reports collecting
+   * information about block locations.
+   * In order to leave safe mode it needs to collect a configurable
+   * percentage called threshold of blocks, which satisfy the minimal 
+   * replication condition.
+   * The minimal replication condition is that each block must have at least
+   * <tt>dfs.namenode.replication.min</tt> replicas.
+   * When the threshold is reached the name node extends safe mode
+   * for a configurable amount of time
+   * to let the remaining data nodes to check in before it
+   * will start replicating missing blocks.
+   * Then the name node leaves safe mode.
+   * <p>
+   * If safe mode is turned on manually using
+   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}
+   * then the name node stays in safe mode until it is manually turned off
+   * using {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_LEAVE,false)}.
+   * Current state of the name node can be verified using
+   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_GET,false)}
+   * <h4>Configuration parameters:</h4>
+   * <tt>dfs.safemode.threshold.pct</tt> is the threshold parameter.<br>
+   * <tt>dfs.safemode.extension</tt> is the safe mode extension parameter.<br>
+   * <tt>dfs.namenode.replication.min</tt> is the minimal replication parameter.
+   * 
+   * <h4>Special cases:</h4>
+   * The name node does not enter safe mode at startup if the threshold is 
+   * set to 0 or if the name space is empty.<br>
+   * If the threshold is set to 1 then all blocks need to have at least 
+   * minimal replication.<br>
+   * If the threshold value is greater than 1 then the name node will not be 
+   * able to turn off safe mode automatically.<br>
+   * Safe mode can always be turned off manually.
+   * 
+   * @param action  <ul> <li>0 leave safe mode;</li>
+   *                <li>1 enter safe mode;</li>
+   *                <li>2 get safe mode state.</li></ul>
+   * @param isChecked If true then action will be done only in ActiveNN.
+   * 
+   * @return <ul><li>0 if the safe mode is OFF or</li> 
+   *         <li>1 if the safe mode is ON.</li></ul>
+   *                   
+   * @throws IOException
+   */
+  @Idempotent
+  public boolean setSafeMode(HdfsConstants.SafeModeAction action, boolean isChecked) 
+      throws IOException;
+
+  /**
+   * Save namespace image.
+   * <p>
+   * Saves current namespace into storage directories and reset edits log.
+   * Requires superuser privilege and safe mode.
+   *
+   * @param timeWindow NameNode does a checkpoint if the latest checkpoint was
+   *                   done beyond the given time period (in seconds).
+   * @param txGap NameNode does a checkpoint if the gap between the latest
+   *              checkpoint and the latest transaction id is greater this gap.
+   * @return whether an extra checkpoint has been done
+   *
+   * @throws IOException if image creation failed.
+   */
+  @AtMostOnce
+  public boolean saveNamespace(long timeWindow, long txGap) throws IOException;
+
+  
+  /**
+   * Roll the edit log.
+   * Requires superuser privileges.
+   * 
+   * @throws AccessControlException if the superuser privilege is violated
+   * @throws IOException if log roll fails
+   * @return the txid of the new segment
+   */
+  @Idempotent
+  public long rollEdits() throws AccessControlException, IOException;
+
+  /**
+   * Enable/Disable restore failed storage.
+   * <p>
+   * sets flag to enable restore of failed storage replicas
+   * 
+   * @throws AccessControlException if the superuser privilege is violated.
+   */
+  @Idempotent
+  public boolean restoreFailedStorage(String arg) 
+      throws AccessControlException, IOException;
+
+  /**
+   * Tells the namenode to reread the hosts and exclude files. 
+   * @throws IOException
+   */
+  @Idempotent
+  public void refreshNodes() throws IOException;
+
+  /**
+   * Finalize previous upgrade.
+   * Remove file system state saved during the upgrade.
+   * The upgrade will become irreversible.
+   * 
+   * @throws IOException
+   */
+  @Idempotent
+  public void finalizeUpgrade() throws IOException;
+
+  /**
+   * Rolling upgrade operations.
+   * @param action either query, prepare or finalize.
+   * @return rolling upgrade information.
+   */
+  @Idempotent
+  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
+      throws IOException;
+
+  /**
+   * @return CorruptFileBlocks, containing a list of corrupt files (with
+   *         duplicates if there is more than one corrupt block in a file)
+   *         and a cookie
+   * @throws IOException
+   *
+   * Each call returns a subset of the corrupt files in the system. To obtain
+   * all corrupt files, call this method repeatedly and each time pass in the
+   * cookie returned from the previous call.
+   */
+  @Idempotent
+  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
+      throws IOException;
+  
+  /**
+   * Dumps namenode data structures into specified file. If the file
+   * already exists, then append.
+   *
+   * @throws IOException
+   */
+  @Idempotent
+  public void metaSave(String filename) throws IOException;
+
+  /**
+   * Tell all datanodes to use a new, non-persistent bandwidth value for
+   * dfs.balance.bandwidthPerSec.
+   *
+   * @param bandwidth Blanacer bandwidth in bytes per second for this datanode.
+   * @throws IOException
+   */
+  @Idempotent
+  public void setBalancerBandwidth(long bandwidth) throws IOException;
+  
+  /**
+   * Get the file info for a specific file or directory.
+   * @param src The string representation of the path to the file
+   *
+   * @return object containing information regarding the file
+   *         or null if file not found
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if the path contains a symlink. 
+   * @throws IOException If an I/O error occurred        
+   */
+  @Idempotent
+  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException;
+  
+  /**
+   * Get the close status of a file
+   * @param src The string representation of the path to the file
+   *
+   * @return return true if file is closed
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if the path contains a symlink.
+   * @throws IOException If an I/O error occurred     
+   */
+  @Idempotent
+  public boolean isFileClosed(String src) throws AccessControlException,
+      FileNotFoundException, UnresolvedLinkException, IOException;
+  
+  /**
+   * Get the file info for a specific file or directory. If the path 
+   * refers to a symlink then the FileStatus of the symlink is returned.
+   * @param src The string representation of the path to the file
+   *
+   * @return object containing information regarding the file
+   *         or null if file not found
+   *
+   * @throws AccessControlException permission denied
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
+   * @throws IOException If an I/O error occurred        
+   */
+  @Idempotent
+  public HdfsFileStatus getFileLinkInfo(String src)
+      throws AccessControlException, UnresolvedLinkException, IOException;
+  
+  /**
+   * Get {@link ContentSummary} rooted at the specified directory.
+   * @param path The string representation of the path
+   *
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>path</code> is not found
+   * @throws UnresolvedLinkException if <code>path</code> contains a symlink. 
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public ContentSummary getContentSummary(String path)
+      throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, IOException;
+
+  /**
+   * Set the quota for a directory.
+   * @param path  The string representation of the path to the directory
+   * @param namespaceQuota Limit on the number of names in the tree rooted 
+   *                       at the directory
+   * @param storagespaceQuota Limit on storage space occupied all the files under
+   *                       this directory.
+   * @param type StorageType that the space quota is intended to be set on.
+   *             It may be null when called by traditional space/namespace quota.
+   *             When type is is not null, the storagespaceQuota parameter is for
+   *             type specified and namespaceQuota must be
+   *             {@link HdfsConstants#QUOTA_DONT_SET}.
+   *
+   * <br><br>
+   *                       
+   * The quota can have three types of values : (1) 0 or more will set 
+   * the quota to that value, (2) {@link HdfsConstants#QUOTA_DONT_SET}  implies
+   * the quota will not be changed, and (3) {@link HdfsConstants#QUOTA_RESET}
+   * implies the quota will be reset. Any other value is a runtime error.
+   * 
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>path</code> is not found
+   * @throws QuotaExceededException if the directory size 
+   *           is greater than the given quota
+   * @throws UnresolvedLinkException if the <code>path</code> contains a symlink. 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+      StorageType type) throws AccessControlException, FileNotFoundException,
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Write all metadata for this file into persistent storage.
+   * The file must be currently open for writing.
+   * @param src The string representation of the path
+   * @param inodeId The inode ID, or GRANDFATHER_INODE_ID if the client is
+   *                too old to support fsync with inode IDs.
+   * @param client The string representation of the client
+   * @param lastBlockLength The length of the last block (under construction) 
+   *                        to be reported to NameNode 
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void fsync(String src, long inodeId, String client,
+                    long lastBlockLength)
+      throws AccessControlException, FileNotFoundException, 
+      UnresolvedLinkException, IOException;
+
+  /**
+   * Sets the modification and access time of the file to the specified time.
+   * @param src The string representation of the path
+   * @param mtime The number of milliseconds since Jan 1, 1970.
+   *              Setting mtime to -1 means that modification time should not be set
+   *              by this call.
+   * @param atime The number of milliseconds since Jan 1, 1970.
+   *              Setting atime to -1 means that access time should not be set
+   *              by this call.
+   *              
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException file <code>src</code> is not found
+   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @Idempotent
+  public void setTimes(String src, long mtime, long atime)
+      throws AccessControlException, FileNotFoundException, 
+      UnresolvedLinkException, SnapshotAccessControlException, IOException;
+
+  /**
+   * Create symlink to a file or directory.
+   * @param target The path of the destination that the
+   *               link points to.
+   * @param link The path of the link being created.
+   * @param dirPerm permissions to use when creating parent directories
+   * @param createParent - if true then missing parent dirs are created
+   *                       if false then parent must exist
+   *
+   * @throws AccessControlException permission denied
+   * @throws FileAlreadyExistsException If file <code>link</code> already exists
+   * @throws FileNotFoundException If parent of <code>link</code> does not exist
+   *           and <code>createParent</code> is false
+   * @throws ParentNotDirectoryException If parent of <code>link</code> is not a
+   *           directory.
+   * @throws UnresolvedLinkException if <code>link</target> contains a symlink. 
+   * @throws SnapshotAccessControlException if path is in RO snapshot
+   * @throws IOException If an I/O error occurred
+   */
+  @AtMostOnce
+  public void createSymlink(String target, String link, FsPermission dirPerm,
+      boolean createParent) throws AccessControlException,
+      FileAlreadyExistsException, FileNotFoundException,
+      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
+      SnapshotAccessControlException, IOException;
+
+  /**
+   * Return the target of the given symlink. If there is an intermediate
+   * symlink in the path (ie a symlink leading up to the final path component)
+   * then the given path is returned with this symlink resolved.
+   *
+   * @param path The path with a link that needs resolution.
+   * @return The path after resolving the first symbolic link in the path.
+   * @throws AccessControlException permission denied
+   * @throws FileNotFoundException If <code>path</code> does not exist
+   * @throws IOException If the given path does not refer to a symlink
+   *           or an I/O error occurred
+   */
+  @Idempotent
+  public String getLinkTarget(String path) throws AccessControlException,
+      FileNotFoundException, IOException; 
+  
+  /**
+   * Get a new generation stamp together with an access token for 
+   * a block under construction
+   * 
+   * This method is called only when a client needs to recover a failed
+   * pipeline or set up a pipeline for appending to a block.
+   * 
+   * @param block a block
+   * @param clientName the name of the client
+   * @return a located block with a new generation stamp and an access token
+   * @throws IOException if any error occurs
+   */
+  @Idempotent
+  public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
+      String clientName) throws IOException;
+
+  /**
+   * Update a pipeline for a block under construction
+   * 
+   * @param clientName the name of the client
+   * @param oldBlock the old block
+   * @param newBlock the new block containing new generation stamp and length
+   * @param newNodes datanodes in the pipeline
+   * @throws IOException if any error occurs
+   */
+  @AtMostOnce
+  public void updatePipeline(String clientName, ExtendedBlock oldBlock, 
+      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
+      throws IOException;
+
+  /**
+   * Get a valid Delegation Token.
+   * 
+   * @param renewer the designated renewer for the token
+   * @return Token<DelegationTokenIdentifier>
+   * @throws IOException
+   */
+  @Idempotent
+  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) 
+      throws IOException;
+
+  /**
+   * Renew an existing delegation token.
+   * 
+   * @param token delegation token obtained earlier
+   * @return the new expiration time
+   * @throws IOException
+   */
+  @Idempotent
+  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException;
+  
+  /**
+   * Cancel an existing delegation token.
+   * 
+   * @param token delegation token
+   * @throws IOException
+   */
+  @Idempotent
+  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+      throws IOException;
+  
+  /**
+   * @return encryption key so a client can encrypt data sent via the
+   *         DataTransferProtocol to/from DataNodes.
+   * @throws IOException
+   */
+  @Idempotent
+  public DataEncryptionKey getDataEncryptionKey() throws IOException;
+  
+  /**
+   * Create a snapshot
+   * @param snapshotRoot the path that is being snapshotted
+   * @param snapshotName name of the snapshot created
+   * @return the snapshot path.
+   * @throws IOException
+   */
+  @AtMostOnce
+  public String createSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException;
+
+  /**
+   * Delete a specific snapshot of a snapshottable directory
+   * @param snapshotRoot  The snapshottable directory
+   * @param snapshotName Name of the snapshot for the snapshottable directory
+   * @throws IOException
+   */
+  @AtMostOnce
+  public void deleteSnapshot(String snapshotRoot, String snapshotName)
+      throws IOException;
+  
+  /**
+   * Rename a snapshot
+   * @param snapshotRoot the directory path where the snapshot was taken 
+   * @param snapshotOldName old name of the snapshot
+   * @param snapshotNewName new name of the snapshot
+   * @throws IOException
+   */
+  @AtMostOnce
+  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
+      String snapshotNewName) throws IOException;
+  
+  /**
+   * Allow snapshot on a directory.
+   * @param snapshotRoot the directory to be snapped
+   * @throws IOException on error
+   */
+  @Idempotent
+  public void allowSnapshot(String snapshotRoot)
+      throws IOException;
+    
+  /**
+   * Disallow snapshot on a directory.
+   * @param snapshotRoot the directory to disallow snapshot
+   * @throws IOException on error
+   */
+  @Idempotent
+  public void disallowSnapshot(String snapshotRoot)
+      throws IOException;
+  
+  /**
+   * Get the difference between two snapshots, or between a snapshot and the
+   * current tree of a directory.
+   * 
+   * @param snapshotRoot
+   *          full path of the directory where snapshots are taken
+   * @param fromSnapshot
+   *          snapshot name of the from point. Null indicates the current
+   *          tree
+   * @param toSnapshot
+   *          snapshot name of the to point. Null indicates the current
+   *          tree.
+   * @return The difference report represented as a {@link SnapshotDiffReport}.
+   * @throws IOException on error
+   */
+  @Idempotent
+  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
+      String fromSnapshot, String toSnapshot) throws IOException;
+
+  /**
+   * Add a CacheDirective to the CacheManager.
+   * 
+   * @param directive A CacheDirectiveInfo to be added
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @return A CacheDirectiveInfo associated with the added directive
+   * @throws IOException if the directive could not be added
+   */
+  @AtMostOnce
+  public long addCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
+
+  /**
+   * Modify a CacheDirective in the CacheManager.
+   * 
+   * @param flags {@link CacheFlag}s to use for this operation.
+   * @throws IOException if the directive could not be modified
+   */
+  @AtMostOnce
+  public void modifyCacheDirective(CacheDirectiveInfo directive,
+      EnumSet<CacheFlag> flags) throws IOException;
+
+  /**
+   * Remove a CacheDirectiveInfo from the CacheManager.
+   * 
+   * @param id of a CacheDirectiveInfo
+   * @throws IOException if the cache directive could not be removed
+   */
+  @AtMostOnce
+  public void removeCacheDirective(long id) throws IOException;
+
+  /**
+   * List the set of cached paths of a cache pool. Incrementally fetches results
+   * from the server.
+   * 
+   * @param prevId The last listed entry ID, or -1 if this is the first call to
+   *               listCacheDirectives.
+   * @param filter Parameters to use to filter the list results, 
+   *               or null to display all directives visible to us.
+   * @return A batch of CacheDirectiveEntry objects.
+   */
+  @Idempotent
+  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
+      long prevId, CacheDirectiveInfo filter) throws IOException;
+
+  /**
+   * Add a new cache pool.
+   * 
+   * @param info Description of the new cache pool
+   * @throws IOException If the request could not be completed.
+   */
+  @AtMostOnce
+  public void addCachePool(CachePoolInfo info) throws IOException;
+
+  /**
+   * Modify an existing cache pool.
+   *
+   * @param req
+   *          The request to modify a cache pool.
+   * @throws IOException 
+   *          If the request could not be completed.
+   */
+  @AtMostOnce
+  public void modifyCachePool(CachePoolInfo req) throws IOException;
+  
+  /**
+   * Remove a cache pool.
+   * 
+   * @param pool name of the cache pool to remove.
+   * @throws IOException if the cache pool did not exist, or could not be
+   *           removed.
+   */
+  @AtMostOnce
+  public void removeCachePool(String pool) throws IOException;
+
+  /**
+   * List the set of cache pools. Incrementally fetches results from the server.
+   * 
+   * @param prevPool name of the last pool listed, or the empty string if this is
+   *          the first invocation of listCachePools
+   * @return A batch of CachePoolEntry objects.
+   */
+  @Idempotent
+  public BatchedEntries<CachePoolEntry> listCachePools(String prevPool)
+      throws IOException;
+
+  /**
+   * Modifies ACL entries of files and directories.  This method can add new ACL
+   * entries or modify the permissions on existing ACL entries.  All existing
+   * ACL entries that are not specified in this call are retained without
+   * changes.  (Modifications are merged into the current ACL.)
+   */
+  @Idempotent
+  public void modifyAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException;
+
+  /**
+   * Removes ACL entries from files and directories.  Other ACL entries are
+   * retained.
+   */
+  @Idempotent
+  public void removeAclEntries(String src, List<AclEntry> aclSpec)
+      throws IOException;
+
+  /**
+   * Removes all default ACL entries from files and directories.
+   */
+  @Idempotent
+  public void removeDefaultAcl(String src) throws IOException;
+
+  /**
+   * Removes all but the base ACL entries of files and directories.  The entries
+   * for user, group, and others are retained for compatibility with permission
+   * bits.
+   */
+  @Idempotent
+  public void removeAcl(String src) throws IOException;
+
+  /**
+   * Fully replaces ACL of files and directories, discarding all existing
+   * entries.
+   */
+  @Idempotent
+  public void setAcl(String src, List<AclEntry> aclSpec) throws IOException;
+
+  /**
+   * Gets the ACLs of files and directories.
+   */
+  @Idempotent
+  public AclStatus getAclStatus(String src) throws IOException;
+  
+  /**
+   * Create an encryption zone
+   */
+  @AtMostOnce
+  public void createEncryptionZone(String src, String keyName)
+    throws IOException;
+
+  /**
+   * Get the encryption zone for a path.
+   */
+  @Idempotent
+  public EncryptionZone getEZForPath(String src)
+    throws IOException;
+
+  /**
+   * Used to implement cursor-based batched listing of {@EncryptionZone}s.
+   *
+   * @param prevId ID of the last item in the previous batch. If there is no
+   *               previous batch, a negative value can be used.
+   * @return Batch of encryption zones.
+   */
+  @Idempotent
+  public BatchedEntries<EncryptionZone> listEncryptionZones(
+      long prevId) throws IOException;
+
+  /**
+   * Set xattr of a file or directory.
+   * The name must be prefixed with the namespace followed by ".". For example,
+   * "user.attr".
+   * <p/>
+   * Refer to the HDFS extended attributes user documentation for details.
+   *
+   * @param src file or directory
+   * @param xAttr <code>XAttr</code> to set
+   * @param flag set flag
+   * @throws IOException
+   */
+  @AtMostOnce
+  public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag) 
+      throws IOException;
+  
+  /**
+   * Get xattrs of a file or directory. Values in xAttrs parameter are ignored.
+   * If xAttrs is null or empty, this is the same as getting all xattrs of the
+   * file or directory.  Only those xattrs for which the logged-in user has
+   * permissions to view are returned.
+   * <p/>
+   * Refer to the HDFS extended attributes user documentation for details.
+   *
+   * @param src file or directory
+   * @param xAttrs xAttrs to get
+   * @return List<XAttr> <code>XAttr</code> list 
+   * @throws IOException
+   */
+  @Idempotent
+  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs) 
+      throws IOException;
+
+  /**
+   * List the xattrs names for a file or directory.
+   * Only the xattr names for which the logged in user has the permissions to
+   * access will be returned.
+   * <p/>
+   * Refer to the HDFS extended attributes user documentation for details.
+   *
+   * @param src file or directory
+   * @return List<XAttr> <code>XAttr</code> list
+   * @throws IOException
+   */
+  @Idempotent
+  public List<XAttr> listXAttrs(String src)
+      throws IOException;
+  
+  /**
+   * Remove xattr of a file or directory.Value in xAttr parameter is ignored.
+   * The name must be prefixed with the namespace followed by ".". For example,
+   * "user.attr".
+   * <p/>
+   * Refer to the HDFS extended attributes user documentation for details.
+   *
+   * @param src file or directory
+   * @param xAttr <code>XAttr</code> to remove
+   * @throws IOException
+   */
+  @AtMostOnce
+  public void removeXAttr(String src, XAttr xAttr) throws IOException;
+
+  /**
+   * Checks if the user can access a path.  The mode specifies which access
+   * checks to perform.  If the requested permissions are granted, then the
+   * method returns normally.  If access is denied, then the method throws an
+   * {@link AccessControlException}.
+   * In general, applications should avoid using this method, due to the risk of
+   * time-of-check/time-of-use race conditions.  The permissions on a file may
+   * change immediately after the access call returns.
+   *
+   * @param path Path to check
+   * @param mode type of access to check
+   * @throws AccessControlException if access is denied
+   * @throws FileNotFoundException if the path does not exist
+   * @throws IOException see specific implementation
+   */
+  @Idempotent
+  public void checkAccess(String path, FsAction mode) throws IOException;
+
+  /**
+   * Get the highest txid the NameNode knows has been written to the edit
+   * log, or -1 if the NameNode's edit log is not yet open for write. Used as
+   * the starting point for the inotify event stream.
+   */
+  @Idempotent
+  public long getCurrentEditLogTxid() throws IOException;
+
+  /**
+   * Get an ordered list of batches of events corresponding to the edit log
+   * transactions for txids equal to or greater than txid.
+   */
+  @Idempotent
+  public EventBatchList getEditsFromTxid(long txid) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb2ccade/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
new file mode 100644
index 0000000..8080bcf
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
@@ -0,0 +1,38 @@
+/**
+ * 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 java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This exception is thrown when the name node is in safe mode.
+ * Client cannot modified namespace until the safe mode is off. 
+ * 
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class SafeModeException extends IOException {
+  private static final long serialVersionUID = 1L;
+  public SafeModeException(String msg) {
+    super(msg);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb2ccade/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index 3f72608..5ce2863 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -490,7 +490,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS = "dfs.short.circuit.shared.memory.watcher.interrupt.check.ms";
   public static final int     DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT = 60000;
   public static final String  DFS_NAMENODE_KEYTAB_FILE_KEY = "dfs.namenode.keytab.file";
-  public static final String  DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY = "dfs.namenode.kerberos.principal";
+  public static final String  DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY =
+      HdfsClientConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
   @Deprecated
   public static final String  DFS_NAMENODE_USER_NAME_KEY = DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
   public static final String  DFS_NAMENODE_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY = "dfs.namenode.kerberos.internal.spnego.principal";


[45/50] [abbrv] hadoop git commit: HADOOP-7139. Allow appending to existing SequenceFiles (Contributed by kanaka kumar avvaru)

Posted by zj...@apache.org.
HADOOP-7139. Allow appending to existing SequenceFiles (Contributed by kanaka kumar avvaru)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e922a278
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e922a278
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e922a278

Branch: refs/heads/YARN-2928
Commit: e922a27884e2e12ed62f60bf65972a3bf555e7a6
Parents: 05d63e6
Author: Vinayakumar B <vi...@apache.org>
Authored: Thu Jun 18 14:39:00 2015 +0530
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:02 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../java/org/apache/hadoop/io/SequenceFile.java |  85 ++++-
 .../hadoop/io/TestSequenceFileAppend.java       | 311 +++++++++++++++++++
 3 files changed, 394 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e922a278/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 4b1b382..3430da6 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -651,6 +651,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11965. determine-flaky-tests needs a summary mode.
     (Yufei Gu via Yongjun Zhang)
 
+    HADOOP-7139. Allow appending to existing SequenceFiles
+    (kanaka kumar avvaru via vinayakumarb)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e922a278/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
index 7a59149..e37e855 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java
@@ -837,7 +837,9 @@ public class SequenceFile {
     DataOutputStream deflateOut = null;
     Metadata metadata = null;
     Compressor compressor = null;
-    
+
+    private boolean appendMode = false;
+
     protected Serializer keySerializer;
     protected Serializer uncompressedValSerializer;
     protected Serializer compressedValSerializer;
@@ -909,6 +911,13 @@ public class SequenceFile {
       }
     }
 
+    static class AppendIfExistsOption extends Options.BooleanOption implements
+        Option {
+      AppendIfExistsOption(boolean value) {
+        super(value);
+      }
+    }
+
     static class KeyClassOption extends Options.ClassOption implements Option {
       KeyClassOption(Class<?> value) {
         super(value);
@@ -958,7 +967,7 @@ public class SequenceFile {
         return codec;
       }
     }
-    
+
     public static Option file(Path value) {
       return new FileOption(value);
     }
@@ -984,6 +993,10 @@ public class SequenceFile {
       return new ReplicationOption(value);
     }
     
+    public static Option appendIfExists(boolean value) {
+      return new AppendIfExistsOption(value);
+    }
+
     public static Option blockSize(long value) {
       return new BlockSizeOption(value);
     }
@@ -1030,6 +1043,8 @@ public class SequenceFile {
       ProgressableOption progressOption = 
         Options.getOption(ProgressableOption.class, opts);
       FileOption fileOption = Options.getOption(FileOption.class, opts);
+      AppendIfExistsOption appendIfExistsOption = Options.getOption(
+          AppendIfExistsOption.class, opts);
       FileSystemOption fsOption = Options.getOption(FileSystemOption.class, opts);
       StreamOption streamOption = Options.getOption(StreamOption.class, opts);
       KeyClassOption keyClassOption = 
@@ -1071,7 +1086,54 @@ public class SequenceFile {
           blockSizeOption.getValue();
         Progressable progress = progressOption == null ? null :
           progressOption.getValue();
-        out = fs.create(p, true, bufferSize, replication, blockSize, progress);
+
+        if (appendIfExistsOption != null && appendIfExistsOption.getValue()
+            && fs.exists(p)) {
+
+          // Read the file and verify header details
+          SequenceFile.Reader reader = new SequenceFile.Reader(conf,
+              SequenceFile.Reader.file(p), new Reader.OnlyHeaderOption());
+          try {
+
+            if (keyClassOption.getValue() != reader.getKeyClass()
+                || valueClassOption.getValue() != reader.getValueClass()) {
+              throw new IllegalArgumentException(
+                  "Key/value class provided does not match the file");
+            }
+
+            if (reader.getVersion() != VERSION[3]) {
+              throw new VersionMismatchException(VERSION[3],
+                  reader.getVersion());
+            }
+
+            if (metadataOption != null) {
+              LOG.info("MetaData Option is ignored during append");
+            }
+            metadataOption = (MetadataOption) SequenceFile.Writer
+                .metadata(reader.getMetadata());
+
+            CompressionOption readerCompressionOption = new CompressionOption(
+                reader.getCompressionType(), reader.getCompressionCodec());
+
+            if (readerCompressionOption.value != compressionTypeOption.value
+                || !readerCompressionOption.codec.getClass().getName()
+                    .equals(compressionTypeOption.codec.getClass().getName())) {
+              throw new IllegalArgumentException(
+                  "Compression option provided does not match the file");
+            }
+
+            sync = reader.getSync();
+
+          } finally {
+            reader.close();
+          }
+
+          out = fs.append(p, bufferSize, progress);
+          this.appendMode = true;
+        } else {
+          out = fs
+              .create(p, true, bufferSize, replication, blockSize, progress);
+        }
       } else {
         out = streamOption.getValue();
       }
@@ -1159,7 +1221,7 @@ public class SequenceFile {
       out.write(sync);                       // write the sync bytes
       out.flush();                           // flush header
     }
-    
+
     /** Initialize. */
     @SuppressWarnings("unchecked")
     void init(Configuration conf, FSDataOutputStream out, boolean ownStream,
@@ -1214,7 +1276,12 @@ public class SequenceFile {
         }
         this.compressedValSerializer.open(deflateOut);
       }
-      writeFileHeader();
+
+      if (appendMode) {
+        sync();
+      } else {
+        writeFileHeader();
+      }
     }
     
     /** Returns the class of keys in this file. */
@@ -2045,6 +2112,14 @@ public class SequenceFile {
     /** Returns the compression codec of data in this file. */
     public CompressionCodec getCompressionCodec() { return codec; }
     
+    private byte[] getSync() {
+      return sync;
+    }
+
+    private byte getVersion() {
+      return version;
+    }
+
     /**
      * Get the compression type for this file.
      * @return the compression type

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e922a278/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFileAppend.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFileAppend.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFileAppend.java
new file mode 100644
index 0000000..4576642
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestSequenceFileAppend.java
@@ -0,0 +1,311 @@
+/**
+ * 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.io;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.SequenceFile.CompressionType;
+import org.apache.hadoop.io.SequenceFile.Reader;
+import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.io.SequenceFile.Writer.Option;
+import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.io.compress.GzipCodec;
+import org.apache.hadoop.io.serializer.JavaSerializationComparator;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestSequenceFileAppend {
+
+  private static Configuration conf;
+  private static FileSystem fs;
+  private static Path ROOT_PATH = new Path(System.getProperty(
+      "test.build.data", "build/test/data"));
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    conf = new Configuration();
+    conf.set("io.serializations",
+        "org.apache.hadoop.io.serializer.JavaSerialization");
+    conf.set("fs.file.impl", "org.apache.hadoop.fs.RawLocalFileSystem");
+    fs = FileSystem.get(conf);
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    fs.close();
+  }
+
+  @Test(timeout = 30000)
+  public void testAppend() throws Exception {
+
+    Path file = new Path(ROOT_PATH, "testseqappend.seq");
+    fs.delete(file, true);
+
+    Text key1 = new Text("Key1");
+    Text value1 = new Text("Value1");
+    Text value2 = new Text("Updated");
+
+    SequenceFile.Metadata metadata = new SequenceFile.Metadata();
+    metadata.set(key1, value1);
+    Writer.Option metadataOption = Writer.metadata(metadata);
+
+    Writer writer = SequenceFile.createWriter(conf,
+        SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class), metadataOption);
+
+    writer.append(1L, "one");
+    writer.append(2L, "two");
+    writer.close();
+
+    verify2Values(file);
+
+    metadata.set(key1, value2);
+
+    writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class),
+        SequenceFile.Writer.appendIfExists(true), metadataOption);
+
+    // Verify the Meta data is not changed
+    assertEquals(value1, writer.metadata.get(key1));
+
+    writer.append(3L, "three");
+    writer.append(4L, "four");
+
+    writer.close();
+
+    verifyAll4Values(file);
+
+    // Verify the Meta data readable after append
+    Reader reader = new Reader(conf, Reader.file(file));
+    assertEquals(value1, reader.getMetadata().get(key1));
+    reader.close();
+
+    // Verify failure if the compression details are different
+    try {
+      Option wrongCompressOption = Writer.compression(CompressionType.RECORD,
+          new GzipCodec());
+
+      writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+          SequenceFile.Writer.keyClass(Long.class),
+          SequenceFile.Writer.valueClass(String.class),
+          SequenceFile.Writer.appendIfExists(true), wrongCompressOption);
+      writer.close();
+      fail("Expected IllegalArgumentException for compression options");
+    } catch (IllegalArgumentException IAE) {
+      // Expected exception. Ignore it
+    }
+
+    try {
+      Option wrongCompressOption = Writer.compression(CompressionType.BLOCK,
+          new DefaultCodec());
+
+      writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+          SequenceFile.Writer.keyClass(Long.class),
+          SequenceFile.Writer.valueClass(String.class),
+          SequenceFile.Writer.appendIfExists(true), wrongCompressOption);
+      writer.close();
+      fail("Expected IllegalArgumentException for compression options");
+    } catch (IllegalArgumentException IAE) {
+      // Expected exception. Ignore it
+    }
+
+    fs.deleteOnExit(file);
+  }
+
+  @Test(timeout = 30000)
+  public void testAppendRecordCompression() throws Exception {
+
+    Path file = new Path(ROOT_PATH, "testseqappendblockcompr.seq");
+    fs.delete(file, true);
+
+    Option compressOption = Writer.compression(CompressionType.RECORD,
+        new GzipCodec());
+    Writer writer = SequenceFile.createWriter(conf,
+        SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class), compressOption);
+
+    writer.append(1L, "one");
+    writer.append(2L, "two");
+    writer.close();
+
+    verify2Values(file);
+
+    writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class),
+        SequenceFile.Writer.appendIfExists(true), compressOption);
+
+    writer.append(3L, "three");
+    writer.append(4L, "four");
+    writer.close();
+
+    verifyAll4Values(file);
+
+    fs.deleteOnExit(file);
+  }
+
+  @Test(timeout = 30000)
+  public void testAppendBlockCompression() throws Exception {
+
+    Path file = new Path(ROOT_PATH, "testseqappendblockcompr.seq");
+    fs.delete(file, true);
+
+    Option compressOption = Writer.compression(CompressionType.BLOCK,
+        new GzipCodec());
+    Writer writer = SequenceFile.createWriter(conf,
+        SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class), compressOption);
+
+    writer.append(1L, "one");
+    writer.append(2L, "two");
+    writer.close();
+
+    verify2Values(file);
+
+    writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class),
+        SequenceFile.Writer.appendIfExists(true), compressOption);
+
+    writer.append(3L, "three");
+    writer.append(4L, "four");
+    writer.close();
+
+    verifyAll4Values(file);
+
+    // Verify failure if the compression details are different or not Provided
+    try {
+      writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+          SequenceFile.Writer.keyClass(Long.class),
+          SequenceFile.Writer.valueClass(String.class),
+          SequenceFile.Writer.appendIfExists(true));
+      writer.close();
+      fail("Expected IllegalArgumentException for compression options");
+    } catch (IllegalArgumentException IAE) {
+      // Expected exception. Ignore it
+    }
+
+    // Verify failure if the compression details are different
+    try {
+      Option wrongCompressOption = Writer.compression(CompressionType.RECORD,
+          new GzipCodec());
+
+      writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+          SequenceFile.Writer.keyClass(Long.class),
+          SequenceFile.Writer.valueClass(String.class),
+          SequenceFile.Writer.appendIfExists(true), wrongCompressOption);
+      writer.close();
+      fail("Expected IllegalArgumentException for compression options");
+    } catch (IllegalArgumentException IAE) {
+      // Expected exception. Ignore it
+    }
+
+    try {
+      Option wrongCompressOption = Writer.compression(CompressionType.BLOCK,
+          new DefaultCodec());
+
+      writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+          SequenceFile.Writer.keyClass(Long.class),
+          SequenceFile.Writer.valueClass(String.class),
+          SequenceFile.Writer.appendIfExists(true), wrongCompressOption);
+      writer.close();
+      fail("Expected IllegalArgumentException for compression options");
+    } catch (IllegalArgumentException IAE) {
+      // Expected exception. Ignore it
+    }
+
+    fs.deleteOnExit(file);
+  }
+
+  @Test(timeout = 30000)
+  public void testAppendSort() throws Exception {
+    Path file = new Path(ROOT_PATH, "testseqappendSort.seq");
+    fs.delete(file, true);
+
+    Path sortedFile = new Path(ROOT_PATH, "testseqappendSort.seq.sort");
+    fs.delete(sortedFile, true);
+
+    SequenceFile.Sorter sorter = new SequenceFile.Sorter(fs,
+        new JavaSerializationComparator<Long>(), Long.class, String.class, conf);
+
+    Option compressOption = Writer.compression(CompressionType.BLOCK,
+        new GzipCodec());
+    Writer writer = SequenceFile.createWriter(conf,
+        SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class), compressOption);
+
+    writer.append(2L, "two");
+    writer.append(1L, "one");
+
+    writer.close();
+
+    writer = SequenceFile.createWriter(conf, SequenceFile.Writer.file(file),
+        SequenceFile.Writer.keyClass(Long.class),
+        SequenceFile.Writer.valueClass(String.class),
+        SequenceFile.Writer.appendIfExists(true), compressOption);
+
+    writer.append(4L, "four");
+    writer.append(3L, "three");
+    writer.close();
+
+    // Sort file after append
+    sorter.sort(file, sortedFile);
+    verifyAll4Values(sortedFile);
+
+    fs.deleteOnExit(file);
+    fs.deleteOnExit(sortedFile);
+  }
+
+  private void verify2Values(Path file) throws IOException {
+    Reader reader = new Reader(conf, Reader.file(file));
+    assertEquals(1L, reader.next((Object) null));
+    assertEquals("one", reader.getCurrentValue((Object) null));
+    assertEquals(2L, reader.next((Object) null));
+    assertEquals("two", reader.getCurrentValue((Object) null));
+    assertNull(reader.next((Object) null));
+    reader.close();
+  }
+
+  private void verifyAll4Values(Path file) throws IOException {
+    Reader reader = new Reader(conf, Reader.file(file));
+    assertEquals(1L, reader.next((Object) null));
+    assertEquals("one", reader.getCurrentValue((Object) null));
+    assertEquals(2L, reader.next((Object) null));
+    assertEquals("two", reader.getCurrentValue((Object) null));
+    assertEquals(3L, reader.next((Object) null));
+    assertEquals("three", reader.getCurrentValue((Object) null));
+    assertEquals(4L, reader.next((Object) null));
+    assertEquals("four", reader.getCurrentValue((Object) null));
+    assertNull(reader.next((Object) null));
+    reader.close();
+  }
+}


[43/50] [abbrv] hadoop git commit: HDFS-8446. Separate safemode related operations in GetBlockLocations(). Contributed by Haohui Mai.

Posted by zj...@apache.org.
HDFS-8446. Separate safemode related operations in GetBlockLocations(). Contributed by Haohui Mai.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/563aa169
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/563aa169
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/563aa169

Branch: refs/heads/YARN-2928
Commit: 563aa1695fb08a4a42482733dc323221c4dab583
Parents: c033209
Author: Haohui Mai <wh...@apache.org>
Authored: Wed Jun 17 16:21:37 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:01 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hdfs/server/namenode/FSDirAttrOp.java       |   3 +-
 .../server/namenode/FSDirStatAndListingOp.java  |  93 +++++++++--
 .../hdfs/server/namenode/FSDirectory.java       |  20 +++
 .../hdfs/server/namenode/FSNamesystem.java      | 153 +++----------------
 .../hdfs/server/namenode/NamenodeFsck.java      |   5 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |  16 +-
 7 files changed, 140 insertions(+), 153 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/563aa169/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 6ef405b..6dfcd18 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -647,6 +647,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8238. Move ClientProtocol to the hdfs-client.
     (Takanobu Asanuma via wheat9)
 
+    HDFS-8446. Separate safemode related operations in GetBlockLocations().
+    (wheat9)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563aa169/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 3b07320..b322b69 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -474,8 +474,7 @@ public class FSDirAttrOp {
 
       // if the last access time update was within the last precision interval, then
       // no need to store access time
-      if (atime <= inodeTime + fsd.getFSNamesystem().getAccessTimePrecision()
-          && !force) {
+      if (atime <= inodeTime + fsd.getAccessTimePrecision() && !force) {
         status =  false;
       } else {
         inode.setAccessTime(atime, latest);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563aa169/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index c636d93..201dabc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -35,6 +36,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectorySnapshottableFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
@@ -43,6 +45,8 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.Arrays;
 
+import static org.apache.hadoop.util.Time.now;
+
 class FSDirStatAndListingOp {
   static DirectoryListing getListingInt(FSDirectory fsd, final String srcArg,
       byte[] startAfter, boolean needLocation) throws IOException {
@@ -137,9 +141,69 @@ class FSDirStatAndListingOp {
     return getContentSummaryInt(fsd, iip);
   }
 
+  /**
+   * Get block locations within the specified range.
+   * @see ClientProtocol#getBlockLocations(String, long, long)
+   * @throws IOException
+   */
+  static GetBlockLocationsResult getBlockLocations(
+      FSDirectory fsd, FSPermissionChecker pc, String src, long offset,
+      long length, boolean needBlockToken) throws IOException {
+    Preconditions.checkArgument(offset >= 0,
+        "Negative offset is not supported. File: " + src);
+    Preconditions.checkArgument(length >= 0,
+        "Negative length is not supported. File: " + src);
+    CacheManager cm = fsd.getFSNamesystem().getCacheManager();
+    BlockManager bm = fsd.getBlockManager();
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    boolean isReservedName = FSDirectory.isReservedRawName(src);
+    fsd.readLock();
+    try {
+      src = fsd.resolvePath(pc, src, pathComponents);
+      final INodesInPath iip = fsd.getINodesInPath(src, true);
+      final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
+      if (fsd.isPermissionEnabled()) {
+        fsd.checkPathAccess(pc, iip, FsAction.READ);
+        fsd.checkUnreadableBySuperuser(pc, inode, iip.getPathSnapshotId());
+      }
+
+      final long fileSize = iip.isSnapshot()
+          ? inode.computeFileSize(iip.getPathSnapshotId())
+          : inode.computeFileSizeNotIncludingLastUcBlock();
+
+      boolean isUc = inode.isUnderConstruction();
+      if (iip.isSnapshot()) {
+        // if src indicates a snapshot file, we need to make sure the returned
+        // blocks do not exceed the size of the snapshot file.
+        length = Math.min(length, fileSize - offset);
+        isUc = false;
+      }
+
+      final FileEncryptionInfo feInfo = isReservedName ? null
+          : fsd.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
+
+      final LocatedBlocks blocks = bm.createLocatedBlocks(
+          inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
+          length, needBlockToken, iip.isSnapshot(), feInfo);
+
+      // Set caching information for the located blocks.
+      for (LocatedBlock lb : blocks.getLocatedBlocks()) {
+        cm.setCachedLocations(lb);
+      }
+
+      final long now = now();
+      boolean updateAccessTime = fsd.isAccessTimeSupported()
+          && !iip.isSnapshot()
+          && now > inode.getAccessTime() + fsd.getAccessTimePrecision();
+      return new GetBlockLocationsResult(updateAccessTime, blocks);
+    } finally {
+      fsd.readUnlock();
+    }
+  }
+
   private static byte getStoragePolicyID(byte inodePolicy, byte parentPolicy) {
-    return inodePolicy != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED ? inodePolicy :
-        parentPolicy;
+    return inodePolicy != HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED
+        ? inodePolicy : parentPolicy;
   }
 
   /**
@@ -294,13 +358,11 @@ class FSDirStatAndListingOp {
       byte policyId = includeStoragePolicy && !i.isSymlink() ?
           i.getStoragePolicyID() :
           HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-      INodeAttributes nodeAttrs = getINodeAttributes(
-          fsd, path, HdfsFileStatus.EMPTY_NAME, i, src.getPathSnapshotId());
-      return createFileStatus(
-          fsd, HdfsFileStatus.EMPTY_NAME,
-          i, nodeAttrs, policyId,
-          src.getPathSnapshotId(),
-          isRawPath, src);
+      INodeAttributes nodeAttrs = getINodeAttributes(fsd, path,
+                                                     HdfsFileStatus.EMPTY_NAME,
+                                                     i, src.getPathSnapshotId());
+      return createFileStatus(fsd, HdfsFileStatus.EMPTY_NAME, i, nodeAttrs,
+                              policyId, src.getPathSnapshotId(), isRawPath, src);
     } finally {
       fsd.readUnlock();
     }
@@ -520,4 +582,17 @@ class FSDirStatAndListingOp {
       fsd.readUnlock();
     }
   }
+
+  static class GetBlockLocationsResult {
+    final boolean updateAccessTime;
+    final LocatedBlocks blocks;
+    boolean updateAccessTime() {
+      return updateAccessTime;
+    }
+    private GetBlockLocationsResult(
+        boolean updateAccessTime, LocatedBlocks blocks) {
+      this.updateAccessTime = updateAccessTime;
+      this.blocks = blocks;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563aa169/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 5b8650e..c807fba 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -80,6 +80,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_KEY;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_ENCRYPTION_ZONE;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.CRYPTO_XATTR_FILE_ENCRYPTION_INFO;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 
 /**
@@ -92,6 +93,7 @@ import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_S
 @InterfaceAudience.Private
 public class FSDirectory implements Closeable {
   static final Logger LOG = LoggerFactory.getLogger(FSDirectory.class);
+
   private static INodeDirectory createRoot(FSNamesystem namesystem) {
     final INodeDirectory r = new INodeDirectory(
         INodeId.ROOT_INODE_ID,
@@ -328,6 +330,9 @@ public class FSDirectory implements Closeable {
   boolean isAccessTimeSupported() {
     return accessTimePrecision > 0;
   }
+  long getAccessTimePrecision() {
+    return accessTimePrecision;
+  }
   boolean isQuotaByStorageTypeEnabled() {
     return quotaByStorageTypeEnabled;
   }
@@ -1550,6 +1555,21 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  void checkUnreadableBySuperuser(
+      FSPermissionChecker pc, INode inode, int snapshotId)
+      throws IOException {
+    if (pc.isSuperUser()) {
+      for (XAttr xattr : FSDirXAttrOp.getXAttrs(this, inode, snapshotId)) {
+        if (XAttrHelper.getPrefixName(xattr).
+            equals(SECURITY_XATTR_UNREADABLE_BY_SUPERUSER)) {
+          throw new AccessControlException(
+              "Access is denied for " + pc.getUser() + " since the superuser "
+              + "is not allowed to perform this operation.");
+        }
+      }
+    }
+  }
+
   HdfsFileStatus getAuditFileInfo(INodesInPath iip)
       throws IOException {
     return (namesystem.isAuditEnabled() && namesystem.isExternalInvocation())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563aa169/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index b97776a..d82da93 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -34,8 +34,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_DEF
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ENCRYPT_DATA_TRANSFER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY;
@@ -87,7 +85,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROU
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
-import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.SECURITY_XATTR_UNREADABLE_BY_SUPERUSER;
+import static org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.*;
 import static org.apache.hadoop.util.Time.now;
 import static org.apache.hadoop.util.Time.monotonicNow;
 
@@ -169,7 +167,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.UnknownCryptoProtocolVersionException;
-import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
@@ -484,9 +481,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private final long minBlockSize;         // minimum block size
   final long maxBlocksPerFile;     // maximum # of blocks per file
 
-  // precision of access times.
-  private final long accessTimePrecision;
-
   /** Lock to protect FSNamesystem. */
   private final FSNamesystemLock fsLock;
 
@@ -800,8 +794,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_DEFAULT);
       this.maxBlocksPerFile = conf.getLong(DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_KEY,
           DFSConfigKeys.DFS_NAMENODE_MAX_BLOCKS_PER_FILE_DEFAULT);
-      this.accessTimePrecision = conf.getLong(DFS_NAMENODE_ACCESSTIME_PRECISION_KEY,
-          DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT);
 
       this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
       
@@ -1631,14 +1623,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return serverDefaults;
   }
 
-  long getAccessTimePrecision() {
-    return accessTimePrecision;
-  }
-
-  private boolean isAccessTimeSupported() {
-    return accessTimePrecision > 0;
-  }
-
   /////////////////////////////////////////////////////////
   //
   // These methods are called by HadoopFS clients
@@ -1689,19 +1673,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     logAuditEvent(true, "setOwner", src, null, auditStat);
   }
 
-  static class GetBlockLocationsResult {
-    final boolean updateAccessTime;
-    final LocatedBlocks blocks;
-    boolean updateAccessTime() {
-      return updateAccessTime;
-    }
-    private GetBlockLocationsResult(
-        boolean updateAccessTime, LocatedBlocks blocks) {
-      this.updateAccessTime = updateAccessTime;
-      this.blocks = blocks;
-    }
-  }
-
   /**
    * Get block locations within the specified range.
    * @see ClientProtocol#getBlockLocations(String, long, long)
@@ -1714,7 +1685,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(OperationCategory.READ);
-      res = getBlockLocations(pc, srcArg, offset, length, true, true);
+      res = FSDirStatAndListingOp.getBlockLocations(
+          dir, pc, srcArg, offset, length, true);
+      if (isInSafeMode()) {
+        for (LocatedBlock b : res.blocks.getLocatedBlocks()) {
+          // if safemode & no block locations yet then throw safemodeException
+          if ((b.getLocations() == null) || (b.getLocations().length == 0)) {
+            SafeModeException se = newSafemodeException(
+                "Zero blocklocations for " + srcArg);
+            if (haEnabled && haContext != null &&
+                haContext.getState().getServiceState() == HAServiceState.ACTIVE) {
+              throw new RetriableException(se);
+            } else {
+              throw se;
+            }
+          }
+        }
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, "open", srcArg);
       throw e;
@@ -1724,7 +1711,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     logAuditEvent(true, "open", srcArg);
 
-    if (res.updateAccessTime()) {
+    if (!isInSafeMode() && res.updateAccessTime()) {
       byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(
           srcArg);
       String src = srcArg;
@@ -1754,7 +1741,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         final INodesInPath iip = dir.getINodesInPath(src, true);
         INode inode = iip.getLastINode();
         boolean updateAccessTime = inode != null &&
-            now > inode.getAccessTime() + getAccessTimePrecision();
+            now > inode.getAccessTime() + dir.getAccessTimePrecision();
         if (!isInSafeMode() && updateAccessTime) {
           boolean changed = FSDirAttrOp.setTimes(dir,
               inode, -1, now, false, iip.getLatestSnapshotId());
@@ -1786,88 +1773,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /**
-   * Get block locations within the specified range.
-   * @see ClientProtocol#getBlockLocations(String, long, long)
-   * @throws IOException
-   */
-  GetBlockLocationsResult getBlockLocations(
-      FSPermissionChecker pc, String src, long offset, long length,
-      boolean needBlockToken, boolean checkSafeMode) throws IOException {
-    if (offset < 0) {
-      throw new HadoopIllegalArgumentException(
-          "Negative offset is not supported. File: " + src);
-    }
-    if (length < 0) {
-      throw new HadoopIllegalArgumentException(
-          "Negative length is not supported. File: " + src);
-    }
-    final GetBlockLocationsResult ret = getBlockLocationsInt(
-        pc, src, offset, length, needBlockToken);
-
-    if (checkSafeMode && isInSafeMode()) {
-      for (LocatedBlock b : ret.blocks.getLocatedBlocks()) {
-        // if safemode & no block locations yet then throw safemodeException
-        if ((b.getLocations() == null) || (b.getLocations().length == 0)) {
-          SafeModeException se = newSafemodeException(
-              "Zero blocklocations for " + src);
-          if (haEnabled && haContext != null &&
-              haContext.getState().getServiceState() == HAServiceState.ACTIVE) {
-            throw new RetriableException(se);
-          } else {
-            throw se;
-          }
-        }
-      }
-    }
-    return ret;
-  }
-
-  private GetBlockLocationsResult getBlockLocationsInt(
-      FSPermissionChecker pc, final String srcArg, long offset, long length,
-      boolean needBlockToken)
-      throws IOException {
-    String src = srcArg;
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
-    src = dir.resolvePath(pc, srcArg, pathComponents);
-    final INodesInPath iip = dir.getINodesInPath(src, true);
-    final INodeFile inode = INodeFile.valueOf(iip.getLastINode(), src);
-    if (isPermissionEnabled) {
-      dir.checkPathAccess(pc, iip, FsAction.READ);
-      checkUnreadableBySuperuser(pc, inode, iip.getPathSnapshotId());
-    }
-
-    final long fileSize = iip.isSnapshot()
-        ? inode.computeFileSize(iip.getPathSnapshotId())
-        : inode.computeFileSizeNotIncludingLastUcBlock();
-    boolean isUc = inode.isUnderConstruction();
-    if (iip.isSnapshot()) {
-      // if src indicates a snapshot file, we need to make sure the returned
-      // blocks do not exceed the size of the snapshot file.
-      length = Math.min(length, fileSize - offset);
-      isUc = false;
-    }
-
-    final FileEncryptionInfo feInfo =
-        FSDirectory.isReservedRawName(srcArg) ? null
-            : dir.getFileEncryptionInfo(inode, iip.getPathSnapshotId(), iip);
-
-    final LocatedBlocks blocks = blockManager.createLocatedBlocks(
-        inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset,
-        length, needBlockToken, iip.isSnapshot(), feInfo);
-
-    // Set caching information for the located blocks.
-    for (LocatedBlock lb : blocks.getLocatedBlocks()) {
-      cacheManager.setCachedLocations(lb);
-    }
-
-    final long now = now();
-    boolean updateAccessTime = isAccessTimeSupported() && !isInSafeMode()
-        && !iip.isSnapshot()
-        && now > inode.getAccessTime() + getAccessTimePrecision();
-    return new GetBlockLocationsResult(updateAccessTime, blocks);
-  }
-
-  /**
    * Moves all the blocks from {@code srcs} and appends them to {@code target}
    * To avoid rollbacks we will verify validity of ALL of the args
    * before we start actual move.
@@ -3912,8 +3817,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     readLock();
     try {
       checkOperation(NameNode.OperationCategory.READ);
-      dl = FSDirStatAndListingOp.getListingInt(dir, src, startAfter,
-          needLocation);
+      dl = getListingInt(dir, src, startAfter, needLocation);
     } catch (AccessControlException e) {
       logAuditEvent(false, "listStatus", src);
       throw e;
@@ -5309,21 +5213,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return new PermissionStatus(fsOwner.getShortUserName(), supergroup, permission);
   }
 
-  private void checkUnreadableBySuperuser(FSPermissionChecker pc,
-      INode inode, int snapshotId)
-      throws IOException {
-    if (pc.isSuperUser()) {
-      for (XAttr xattr : FSDirXAttrOp.getXAttrs(dir, inode, snapshotId)) {
-        if (XAttrHelper.getPrefixName(xattr).
-            equals(SECURITY_XATTR_UNREADABLE_BY_SUPERUSER)) {
-          throw new AccessControlException("Access is denied for " +
-              pc.getUser() + " since the superuser is not allowed to " +
-              "perform this operation.");
-        }
-      }
-    }
-  }
-
   @Override
   public void checkSuperuserPrivilege()
       throws AccessControlException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563aa169/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index f67d25a..7d4cd7e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -483,8 +483,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     final FSNamesystem fsn = namenode.getNamesystem();
     fsn.readLock();
     try {
-      blocks = fsn.getBlockLocations(
-          fsn.getPermissionChecker(), path, 0, fileLen, false, false)
+      blocks = FSDirStatAndListingOp.getBlockLocations(
+          fsn.getFSDirectory(), fsn.getPermissionChecker(),
+          path, 0, fileLen, false)
           .blocks;
     } catch (FileNotFoundException fnfe) {
       blocks = null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/563aa169/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git 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
index eabd0c8..8818f17 100644
--- 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
@@ -1159,20 +1159,21 @@ public class TestFsck {
     Configuration conf = new Configuration();
     NameNode namenode = mock(NameNode.class);
     NetworkTopology nettop = mock(NetworkTopology.class);
-    Map<String,String[]> pmap = new HashMap<String, String[]>();
+    Map<String,String[]> pmap = new HashMap<>();
     Writer result = new StringWriter();
     PrintWriter out = new PrintWriter(result, true);
     InetAddress remoteAddress = InetAddress.getLocalHost();
     FSNamesystem fsName = mock(FSNamesystem.class);
+    FSDirectory fsd = mock(FSDirectory.class);
     BlockManager blockManager = mock(BlockManager.class);
     DatanodeManager dnManager = mock(DatanodeManager.class);
+    INodesInPath iip = mock(INodesInPath.class);
 
     when(namenode.getNamesystem()).thenReturn(fsName);
-    when(fsName.getBlockLocations(any(FSPermissionChecker.class), anyString(),
-                                  anyLong(), anyLong(),
-                                  anyBoolean(), anyBoolean()))
-        .thenThrow(new FileNotFoundException());
     when(fsName.getBlockManager()).thenReturn(blockManager);
+    when(fsName.getFSDirectory()).thenReturn(fsd);
+    when(fsd.getFSNamesystem()).thenReturn(fsName);
+    when(fsd.getINodesInPath(anyString(), anyBoolean())).thenReturn(iip);
     when(blockManager.getDatanodeManager()).thenReturn(dnManager);
 
     NamenodeFsck fsck = new NamenodeFsck(conf, namenode, nettop, pmap, out,
@@ -1190,8 +1191,7 @@ public class TestFsck {
     String owner = "foo";
     String group = "bar";
     byte [] symlink = null;
-    byte [] path = new byte[128];
-    path = DFSUtil.string2Bytes(pathString);
+    byte [] path = DFSUtil.string2Bytes(pathString);
     long fileId = 312321L;
     int numChildren = 1;
     byte storagePolicy = 0;
@@ -1204,7 +1204,7 @@ public class TestFsck {
     try {
       fsck.check(pathString, file, res);
     } catch (Exception e) {
-      fail("Unexpected exception "+ e.getMessage());
+      fail("Unexpected exception " + e.getMessage());
     }
     assertTrue(res.toString().contains("HEALTHY"));
   }


[37/50] [abbrv] hadoop git commit: HDFS-8608. Merge HDFS-7912 to trunk and branch-2 (track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks). Contributed by Zhe Zhang.

Posted by zj...@apache.org.
HDFS-8608. Merge HDFS-7912 to trunk and branch-2 (track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks). Contributed by Zhe Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5a4ccbd0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5a4ccbd0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5a4ccbd0

Branch: refs/heads/YARN-2928
Commit: 5a4ccbd0af7c2ba89c53ce77e0de7c12b5daa7a7
Parents: c7ed62f
Author: Andrew Wang <wa...@apache.org>
Authored: Wed Jun 17 08:05:44 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:00 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  4 ++
 .../server/blockmanagement/BlockManager.java    | 54 ++++++++-------
 .../PendingReplicationBlocks.java               | 51 +++++++-------
 .../blockmanagement/UnderReplicatedBlocks.java  | 57 ++++++++--------
 .../hdfs/server/namenode/FSDirAttrOp.java       |  8 +--
 .../hdfs/server/namenode/FSNamesystem.java      | 21 +++---
 .../hadoop/hdfs/server/namenode/INode.java      | 12 ++--
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  4 +-
 .../hdfs/server/namenode/NamenodeFsck.java      | 58 ++++++++--------
 .../hadoop/hdfs/server/namenode/SafeMode.java   |  4 +-
 .../blockmanagement/BlockManagerTestUtil.java   |  5 +-
 .../blockmanagement/TestBlockManager.java       |  8 +--
 .../server/blockmanagement/TestNodeCount.java   |  3 +-
 .../TestOverReplicatedBlocks.java               |  5 +-
 .../blockmanagement/TestPendingReplication.java | 27 +++++---
 .../TestRBWBlockInvalidation.java               |  4 +-
 .../blockmanagement/TestReplicationPolicy.java  | 70 +++++++++++---------
 .../TestUnderReplicatedBlockQueues.java         | 16 +++--
 .../datanode/TestReadOnlySharedStorage.java     | 11 +--
 .../namenode/TestProcessCorruptBlocks.java      |  5 +-
 20 files changed, 231 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 07cd4a8..a01446a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -640,6 +640,10 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-7164. Feature documentation for HDFS-6581. (Arpit Agarwal)
 
+    HDFS-9608. Merge HDFS-7912 to trunk and branch-2 (track BlockInfo instead
+    of Block in UnderReplicatedBlocks and PendingReplicationBlocks).
+    (Zhe Zhang via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index ebc9017..824801f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -588,7 +588,7 @@ public class BlockManager {
   /**
    * @return true if the block has minimum replicas
    */
-  public boolean checkMinReplication(Block block) {
+  public boolean checkMinReplication(BlockInfo block) {
     return (countNodes(block).liveReplicas() >= minReplication);
   }
 
@@ -1310,7 +1310,7 @@ public class BlockManager {
    * @return number of blocks scheduled for replication during this iteration.
    */
   int computeReplicationWork(int blocksToProcess) {
-    List<List<Block>> blocksToReplicate = null;
+    List<List<BlockInfo>> blocksToReplicate = null;
     namesystem.writeLock();
     try {
       // Choose the blocks to be replicated
@@ -1328,7 +1328,7 @@ public class BlockManager {
    * @return the number of blocks scheduled for replication
    */
   @VisibleForTesting
-  int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
+  int computeReplicationWorkForBlocks(List<List<BlockInfo>> blocksToReplicate) {
     int requiredReplication, numEffectiveReplicas;
     List<DatanodeDescriptor> containingNodes;
     DatanodeDescriptor srcNode;
@@ -1342,7 +1342,7 @@ public class BlockManager {
     try {
       synchronized (neededReplications) {
         for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
-          for (Block block : blocksToReplicate.get(priority)) {
+          for (BlockInfo block : blocksToReplicate.get(priority)) {
             // block should belong to a file
             bc = blocksMap.getBlockCollection(block);
             // abandoned block or block reopened for append
@@ -1426,7 +1426,7 @@ public class BlockManager {
         }
 
         synchronized (neededReplications) {
-          Block block = rw.block;
+          BlockInfo block = rw.block;
           int priority = rw.priority;
           // Recheck since global lock was released
           // block should belong to a file
@@ -1688,7 +1688,7 @@ public class BlockManager {
    * and put them back into the neededReplication queue
    */
   private void processPendingReplications() {
-    Block[] timedOutItems = pendingReplications.getTimedOutBlocks();
+    BlockInfo[] timedOutItems = pendingReplications.getTimedOutBlocks();
     if (timedOutItems != null) {
       namesystem.writeLock();
       try {
@@ -2895,13 +2895,13 @@ public class BlockManager {
   
   /** Set replication for the blocks. */
   public void setReplication(final short oldRepl, final short newRepl,
-      final String src, final Block... blocks) {
+      final String src, final BlockInfo... blocks) {
     if (newRepl == oldRepl) {
       return;
     }
 
     // update needReplication priority queues
-    for(Block b : blocks) {
+    for(BlockInfo b : blocks) {
       updateNeededReplications(b, 0, newRepl-oldRepl);
     }
       
@@ -2909,7 +2909,7 @@ public class BlockManager {
       // old replication > the new one; need to remove copies
       LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl
           + " for " + src);
-      for(Block b : blocks) {
+      for(BlockInfo b : blocks) {
         processOverReplicatedBlock(b, newRepl, null, null);
       }
     } else { // replication factor is increased
@@ -3092,7 +3092,8 @@ public class BlockManager {
     blockLog.debug("BLOCK* removeStoredBlock: {} from {}", block, node);
     assert (namesystem.hasWriteLock());
     {
-      if (!blocksMap.removeNode(block, node)) {
+      BlockInfo storedBlock = getStoredBlock(block);
+      if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) {
         blockLog.debug("BLOCK* removeStoredBlock: {} has already been" +
             " removed from node {}", block, node);
         return;
@@ -3106,8 +3107,8 @@ public class BlockManager {
       //
       BlockCollection bc = blocksMap.getBlockCollection(block);
       if (bc != null) {
-        namesystem.decrementSafeBlockCount(block);
-        updateNeededReplications(block, -1, 0);
+        namesystem.decrementSafeBlockCount(storedBlock);
+        updateNeededReplications(storedBlock, -1, 0);
       }
 
       //
@@ -3181,7 +3182,10 @@ public class BlockManager {
     //
     // Modify the blocks->datanode map and node's map.
     //
-    pendingReplications.decrement(block, node);
+    BlockInfo storedBlock = getStoredBlock(block);
+    if (storedBlock != null) {
+      pendingReplications.decrement(getStoredBlock(block), node);
+    }
     processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED,
         delHintNode);
   }
@@ -3293,7 +3297,7 @@ public class BlockManager {
    * Return the number of nodes hosting a given block, grouped
    * by the state of those replicas.
    */
-  public NumberReplicas countNodes(Block b) {
+  public NumberReplicas countNodes(BlockInfo b) {
     int decommissioned = 0;
     int decommissioning = 0;
     int live = 0;
@@ -3326,12 +3330,12 @@ public class BlockManager {
   }
 
   /** 
-   * Simpler, faster form of {@link #countNodes(Block)} that only returns the number
+   * Simpler, faster form of {@link #countNodes} that only returns the number
    * of live nodes.  If in startup safemode (or its 30-sec extension period),
    * then it gains speed by ignoring issues of excess replicas or nodes
    * that are decommissioned or in process of becoming decommissioned.
-   * If not in startup, then it calls {@link #countNodes(Block)} instead.
-   * 
+   * If not in startup, then it calls {@link #countNodes} instead.
+   *
    * @param b - the block being tested
    * @return count of live nodes for this block
    */
@@ -3360,10 +3364,10 @@ public class BlockManager {
     if (!namesystem.isPopulatingReplQueues()) {
       return;
     }
-    final Iterator<? extends Block> it = srcNode.getBlockIterator();
+    final Iterator<BlockInfo> it = srcNode.getBlockIterator();
     int numOverReplicated = 0;
     while(it.hasNext()) {
-      final Block block = it.next();
+      final BlockInfo block = it.next();
       BlockCollection bc = blocksMap.getBlockCollection(block);
       short expectedReplication = bc.getPreferredBlockReplication();
       NumberReplicas num = countNodes(block);
@@ -3427,7 +3431,7 @@ public class BlockManager {
     return blocksMap.size();
   }
 
-  public void removeBlock(Block block) {
+  public void removeBlock(BlockInfo block) {
     assert namesystem.hasWriteLock();
     // No need to ACK blocks that are being removed entirely
     // from the namespace, since the removal of the associated
@@ -3448,7 +3452,7 @@ public class BlockManager {
   }
 
   /** updates a block in under replication queue */
-  private void updateNeededReplications(final Block block,
+  private void updateNeededReplications(final BlockInfo block,
       final int curReplicasDelta, int expectedReplicasDelta) {
     namesystem.writeLock();
     try {
@@ -3480,7 +3484,7 @@ public class BlockManager {
    */
   public void checkReplication(BlockCollection bc) {
     final short expected = bc.getPreferredBlockReplication();
-    for (Block block : bc.getBlocks()) {
+    for (BlockInfo block : bc.getBlocks()) {
       final NumberReplicas n = countNodes(block);
       if (isNeededReplication(block, expected, n.liveReplicas())) { 
         neededReplications.add(block, n.liveReplicas(),
@@ -3682,7 +3686,7 @@ public class BlockManager {
   /**
    * Return an iterator over the set of blocks for which there are no replicas.
    */
-  public Iterator<Block> getCorruptReplicaBlockIterator() {
+  public Iterator<BlockInfo> getCorruptReplicaBlockIterator() {
     return neededReplications.iterator(
         UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
   }
@@ -3807,7 +3811,7 @@ public class BlockManager {
 
   private static class ReplicationWork {
 
-    private final Block block;
+    private final BlockInfo block;
     private final BlockCollection bc;
 
     private final DatanodeDescriptor srcNode;
@@ -3818,7 +3822,7 @@ public class BlockManager {
     private DatanodeStorageInfo targets[];
     private final int priority;
 
-    public ReplicationWork(Block block,
+    public ReplicationWork(BlockInfo block,
         BlockCollection bc,
         DatanodeDescriptor srcNode,
         List<DatanodeDescriptor> containingNodes,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
index 796b878..04232cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
@@ -23,6 +23,7 @@ import java.io.PrintWriter;
 import java.sql.Time;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -46,8 +47,8 @@ import org.slf4j.Logger;
 class PendingReplicationBlocks {
   private static final Logger LOG = BlockManager.LOG;
 
-  private final Map<Block, PendingBlockInfo> pendingReplications;
-  private final ArrayList<Block> timedOutItems;
+  private final Map<BlockInfo, PendingBlockInfo> pendingReplications;
+  private final ArrayList<BlockInfo> timedOutItems;
   Daemon timerThread = null;
   private volatile boolean fsRunning = true;
 
@@ -62,8 +63,8 @@ class PendingReplicationBlocks {
     if ( timeoutPeriod > 0 ) {
       this.timeout = timeoutPeriod;
     }
-    pendingReplications = new HashMap<Block, PendingBlockInfo>();
-    timedOutItems = new ArrayList<Block>();
+    pendingReplications = new HashMap<>();
+    timedOutItems = new ArrayList<>();
   }
 
   void start() {
@@ -76,7 +77,7 @@ class PendingReplicationBlocks {
    * @param block The corresponding block
    * @param targets The DataNodes where replicas of the block should be placed
    */
-  void increment(Block block, DatanodeDescriptor[] targets) {
+  void increment(BlockInfo block, DatanodeDescriptor[] targets) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found == null) {
@@ -93,9 +94,9 @@ class PendingReplicationBlocks {
    * Decrement the number of pending replication requests
    * for this block.
    * 
-   * @param The DataNode that finishes the replication
+   * @param dn The DataNode that finishes the replication
    */
-  void decrement(Block block, DatanodeDescriptor dn) {
+  void decrement(BlockInfo block, DatanodeDescriptor dn) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found != null) {
@@ -115,7 +116,7 @@ class PendingReplicationBlocks {
    * @param block The given block whose pending replication requests need to be
    *              removed
    */
-  void remove(Block block) {
+  void remove(BlockInfo block) {
     synchronized (pendingReplications) {
       pendingReplications.remove(block);
     }
@@ -138,7 +139,7 @@ class PendingReplicationBlocks {
   /**
    * How many copies of this block is pending replication?
    */
-  int getNumReplicas(Block block) {
+  int getNumReplicas(BlockInfo block) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found != null) {
@@ -153,13 +154,13 @@ class PendingReplicationBlocks {
    * replication requests. Returns null if no blocks have
    * timed out.
    */
-  Block[] getTimedOutBlocks() {
+  BlockInfo[] getTimedOutBlocks() {
     synchronized (timedOutItems) {
       if (timedOutItems.size() <= 0) {
         return null;
       }
-      Block[] blockList = timedOutItems.toArray(
-          new Block[timedOutItems.size()]);
+      BlockInfo[] blockList = timedOutItems.toArray(
+          new BlockInfo[timedOutItems.size()]);
       timedOutItems.clear();
       return blockList;
     }
@@ -179,7 +180,7 @@ class PendingReplicationBlocks {
     PendingBlockInfo(DatanodeDescriptor[] targets) {
       this.timeStamp = monotonicNow();
       this.targets = targets == null ? new ArrayList<DatanodeDescriptor>()
-          : new ArrayList<DatanodeDescriptor>(Arrays.asList(targets));
+          : new ArrayList<>(Arrays.asList(targets));
     }
 
     long getTimeStamp() {
@@ -192,9 +193,7 @@ class PendingReplicationBlocks {
 
     void incrementReplicas(DatanodeDescriptor... newTargets) {
       if (newTargets != null) {
-        for (DatanodeDescriptor dn : newTargets) {
-          targets.add(dn);
-        }
+        Collections.addAll(targets, newTargets);
       }
     }
 
@@ -232,17 +231,17 @@ class PendingReplicationBlocks {
      */
     void pendingReplicationCheck() {
       synchronized (pendingReplications) {
-        Iterator<Map.Entry<Block, PendingBlockInfo>> iter =
+        Iterator<Map.Entry<BlockInfo, PendingBlockInfo>> iter =
                                     pendingReplications.entrySet().iterator();
         long now = monotonicNow();
         if(LOG.isDebugEnabled()) {
           LOG.debug("PendingReplicationMonitor checking Q");
         }
         while (iter.hasNext()) {
-          Map.Entry<Block, PendingBlockInfo> entry = iter.next();
+          Map.Entry<BlockInfo, PendingBlockInfo> entry = iter.next();
           PendingBlockInfo pendingBlock = entry.getValue();
           if (now > pendingBlock.getTimeStamp() + timeout) {
-            Block block = entry.getKey();
+            BlockInfo block = entry.getKey();
             synchronized (timedOutItems) {
               timedOutItems.add(block);
             }
@@ -275,16 +274,14 @@ class PendingReplicationBlocks {
     synchronized (pendingReplications) {
       out.println("Metasave: Blocks being replicated: " +
                   pendingReplications.size());
-      Iterator<Map.Entry<Block, PendingBlockInfo>> iter =
-                                  pendingReplications.entrySet().iterator();
-      while (iter.hasNext()) {
-        Map.Entry<Block, PendingBlockInfo> entry = iter.next();
+      for (Map.Entry<BlockInfo, PendingBlockInfo> entry :
+          pendingReplications.entrySet()) {
         PendingBlockInfo pendingBlock = entry.getValue();
         Block block = entry.getKey();
-        out.println(block + 
-                    " StartTime: " + new Time(pendingBlock.timeStamp) +
-                    " NumReplicaInProgress: " + 
-                    pendingBlock.getNumReplicas());
+        out.println(block +
+            " StartTime: " + new Time(pendingBlock.timeStamp) +
+            " NumReplicaInProgress: " +
+            pendingBlock.getNumReplicas());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
index 1daa0ee..000416e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
@@ -35,7 +34,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  *
  * <p/>
  * The policy for choosing which priority to give added blocks
- * is implemented in {@link #getPriority(Block, int, int, int)}.
+ * is implemented in {@link #getPriority(int, int, int)}.
  * </p>
  * <p>The queue order is as follows:</p>
  * <ol>
@@ -62,7 +61,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  *   blocks that are not corrupt higher priority.</li>
  * </ol>
  */
-class UnderReplicatedBlocks implements Iterable<Block> {
+class UnderReplicatedBlocks implements Iterable<BlockInfo> {
   /** The total number of queues : {@value} */
   static final int LEVEL = 5;
   /** The queue with the highest priority: {@value} */
@@ -78,8 +77,8 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /** The queue for corrupt blocks: {@value} */
   static final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
   /** the queues themselves */
-  private final List<LightWeightLinkedSet<Block>> priorityQueues
-      = new ArrayList<LightWeightLinkedSet<Block>>(LEVEL);
+  private final List<LightWeightLinkedSet<BlockInfo>> priorityQueues
+      = new ArrayList<>(LEVEL);
 
   /** The number of corrupt blocks with replication factor 1 */
   private int corruptReplOneBlocks = 0;
@@ -87,7 +86,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /** Create an object. */
   UnderReplicatedBlocks() {
     for (int i = 0; i < LEVEL; i++) {
-      priorityQueues.add(new LightWeightLinkedSet<Block>());
+      priorityQueues.add(new LightWeightLinkedSet<BlockInfo>());
     }
   }
 
@@ -131,8 +130,8 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** Check if a block is in the neededReplication queue */
-  synchronized boolean contains(Block block) {
-    for(LightWeightLinkedSet<Block> set : priorityQueues) {
+  synchronized boolean contains(BlockInfo block) {
+    for(LightWeightLinkedSet<BlockInfo> set : priorityQueues) {
       if (set.contains(block)) {
         return true;
       }
@@ -141,13 +140,11 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** Return the priority of a block
-   * @param block a under replicated block
    * @param curReplicas current number of replicas of the block
    * @param expectedReplicas expected number of replicas of the block
    * @return the priority for the blocks, between 0 and ({@link #LEVEL}-1)
    */
-  private int getPriority(Block block,
-                          int curReplicas, 
+  private int getPriority(int curReplicas,
                           int decommissionedReplicas,
                           int expectedReplicas) {
     assert curReplicas >= 0 : "Negative replicas!";
@@ -183,12 +180,12 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param expectedReplicas expected number of replicas of the block
    * @return true if the block was added to a queue.
    */
-  synchronized boolean add(Block block,
-                           int curReplicas, 
+  synchronized boolean add(BlockInfo block,
+                           int curReplicas,
                            int decomissionedReplicas,
                            int expectedReplicas) {
     assert curReplicas >= 0 : "Negative replicas!";
-    int priLevel = getPriority(block, curReplicas, decomissionedReplicas,
+    int priLevel = getPriority(curReplicas, decomissionedReplicas,
                                expectedReplicas);
     if(priorityQueues.get(priLevel).add(block)) {
       if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
@@ -207,11 +204,11 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** remove a block from a under replication queue */
-  synchronized boolean remove(Block block, 
-                              int oldReplicas, 
+  synchronized boolean remove(BlockInfo block,
+                              int oldReplicas,
                               int decommissionedReplicas,
                               int oldExpectedReplicas) {
-    int priLevel = getPriority(block, oldReplicas, 
+    int priLevel = getPriority(oldReplicas,
                                decommissionedReplicas,
                                oldExpectedReplicas);
     boolean removedBlock = remove(block, priLevel);
@@ -241,8 +238,8 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param priLevel expected privilege level
    * @return true if the block was found and removed from one of the priority queues
    */
-  boolean remove(Block block, int priLevel) {
-    if(priLevel >= 0 && priLevel < LEVEL 
+  boolean remove(BlockInfo block, int priLevel) {
+    if(priLevel >= 0 && priLevel < LEVEL
         && priorityQueues.get(priLevel).remove(block)) {
       NameNode.blockStateChangeLog.debug(
         "BLOCK* NameSystem.UnderReplicationBlock.remove: Removing block {}" +
@@ -279,14 +276,16 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param curReplicasDelta the change in the replicate count from before
    * @param expectedReplicasDelta the change in the expected replica count from before
    */
-  synchronized void update(Block block, int curReplicas,
+  synchronized void update(BlockInfo block, int curReplicas,
                            int decommissionedReplicas,
                            int curExpectedReplicas,
                            int curReplicasDelta, int expectedReplicasDelta) {
     int oldReplicas = curReplicas-curReplicasDelta;
     int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
-    int curPri = getPriority(block, curReplicas, decommissionedReplicas, curExpectedReplicas);
-    int oldPri = getPriority(block, oldReplicas, decommissionedReplicas, oldExpectedReplicas);
+    int curPri = getPriority(curReplicas, decommissionedReplicas,
+        curExpectedReplicas);
+    int oldPri = getPriority(oldReplicas, decommissionedReplicas,
+        oldExpectedReplicas);
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " + 
         block +
@@ -336,12 +335,12 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @return Return a list of block lists to be replicated. The block list index
    *         represents its replication priority.
    */
-  public synchronized List<List<Block>> chooseUnderReplicatedBlocks(
+  public synchronized List<List<BlockInfo>> chooseUnderReplicatedBlocks(
       int blocksToProcess) {
     // initialize data structure for the return value
-    List<List<Block>> blocksToReplicate = new ArrayList<List<Block>>(LEVEL);
+    List<List<BlockInfo>> blocksToReplicate = new ArrayList<>(LEVEL);
     for (int i = 0; i < LEVEL; i++) {
-      blocksToReplicate.add(new ArrayList<Block>());
+      blocksToReplicate.add(new ArrayList<BlockInfo>());
     }
 
     if (size() == 0) { // There are no blocks to collect.
@@ -364,7 +363,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
       // Loop through all remaining blocks in the list.
       while (blockCount < blocksToProcess
           && neededReplicationsIterator.hasNext()) {
-        Block block = neededReplicationsIterator.next();
+        BlockInfo block = neededReplicationsIterator.next();
         blocksToReplicate.get(priority).add(block);
         blockCount++;
       }
@@ -396,10 +395,10 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /**
    * An iterator over blocks.
    */
-  class BlockIterator implements Iterator<Block> {
+  class BlockIterator implements Iterator<BlockInfo> {
     private int level;
     private boolean isIteratorForLevel = false;
-    private final List<Iterator<Block>> iterators = new ArrayList<Iterator<Block>>();
+    private final List<Iterator<BlockInfo>> iterators = new ArrayList<>();
 
     /**
      * Construct an iterator over all queues.
@@ -431,7 +430,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
     }
 
     @Override
-    public Block next() {
+    public BlockInfo next() {
       if (isIteratorForLevel) {
         return iterators.get(0).next();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index 879738d..3b07320 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -25,12 +25,12 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.util.EnumCounters;
@@ -148,8 +148,8 @@ public class FSDirAttrOp {
       }
 
       final short[] blockRepls = new short[2]; // 0: old, 1: new
-      final Block[] blocks = unprotectedSetReplication(fsd, src, replication,
-                                                       blockRepls);
+      final BlockInfo[] blocks = unprotectedSetReplication(fsd, src,
+          replication, blockRepls);
       isFile = blocks != null;
       if (isFile) {
         fsd.getEditLog().logSetReplication(src, replication);
@@ -375,7 +375,7 @@ public class FSDirAttrOp {
     }
   }
 
-  static Block[] unprotectedSetReplication(
+  static BlockInfo[] unprotectedSetReplication(
       FSDirectory fsd, String src, short replication, short[] blockRepls)
       throws QuotaExceededException, UnresolvedLinkException,
              SnapshotAccessControlException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 40c71ea..b97776a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3170,8 +3170,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *          of blocks that need to be removed from blocksMap
    */
   void removeBlocks(BlocksMapUpdateInfo blocks) {
-    List<Block> toDeleteList = blocks.getToDeleteList();
-    Iterator<Block> iter = toDeleteList.iterator();
+    List<BlockInfo> toDeleteList = blocks.getToDeleteList();
+    Iterator<BlockInfo> iter = toDeleteList.iterator();
     while (iter.hasNext()) {
       writeLock();
       try {
@@ -3227,12 +3227,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean trackBlockCounts = isSafeModeTrackingBlocks();
     int numRemovedComplete = 0, numRemovedSafe = 0;
 
-    for (Block b : blocks.getToDeleteList()) {
+    for (BlockInfo b : blocks.getToDeleteList()) {
       if (trackBlockCounts) {
-        BlockInfo bi = getStoredBlock(b);
-        if (bi.isComplete()) {
+        if (b.isComplete()) {
           numRemovedComplete++;
-          if (bi.numNodes() >= blockManager.minReplication) {
+          if (blockManager.checkMinReplication(b)) {
             numRemovedSafe++;
           }
         }
@@ -4151,7 +4150,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       boolean changed = false;
       writeLock();
       try {
-        final Iterator<Block> it = blockManager.getCorruptReplicaBlockIterator();
+        final Iterator<BlockInfo> it =
+            blockManager.getCorruptReplicaBlockIterator();
 
         while (it.hasNext()) {
           Block b = it.next();
@@ -5093,7 +5093,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override
-  public void decrementSafeBlockCount(Block b) {
+  public void decrementSafeBlockCount(BlockInfo b) {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
@@ -5918,7 +5918,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
       // print a limited # of corrupt files per call
 
-      final Iterator<Block> blkIterator = blockManager.getCorruptReplicaBlockIterator();
+      final Iterator<BlockInfo> blkIterator =
+          blockManager.getCorruptReplicaBlockIterator();
 
       int skip = getIntCookie(cookieTab[0]);
       for (int i = 0; i < skip && blkIterator.hasNext(); i++) {
@@ -5926,7 +5927,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
 
       while (blkIterator.hasNext()) {
-        Block blk = blkIterator.next();
+        BlockInfo blk = blkIterator.next();
         final INode inode = (INode)blockManager.getBlockCollection(blk);
         skip++;
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index cf38fa5..8a2d17a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -34,9 +34,9 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
@@ -950,8 +950,8 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     /**
      * The list of blocks that need to be removed from blocksMap
      */
-    private final List<Block> toDeleteList;
-    
+    private final List<BlockInfo> toDeleteList;
+
     public BlocksMapUpdateInfo() {
       toDeleteList = new ChunkedArrayList<>();
     }
@@ -959,7 +959,7 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     /**
      * @return The list of blocks that need to be removed from blocksMap
      */
-    public List<Block> getToDeleteList() {
+    public List<BlockInfo> getToDeleteList() {
       return toDeleteList;
     }
     
@@ -968,12 +968,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
      * {@link BlocksMapUpdateInfo#toDeleteList}
      * @param toDelete the to-be-deleted block
      */
-    public void addDeleteBlock(Block toDelete) {
+    public void addDeleteBlock(BlockInfo toDelete) {
       assert toDelete != null : "toDelete is null";
       toDeleteList.add(toDelete);
     }
 
-    public void removeDeleteBlock(Block block) {
+    public void removeDeleteBlock(BlockInfo block) {
       assert block != null : "block is null";
       toDeleteList.remove(block);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 4590eec..48879d7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -904,8 +904,8 @@ public class INodeFile extends INodeWithAdditionalFields
         getDiffs().findEarlierSnapshotBlocks(snapshotId);
     if(snapshotBlocks == null)
       return;
-    List<Block> toDelete = collectedBlocks.getToDeleteList();
-    for(Block blk : snapshotBlocks) {
+    List<BlockInfo> toDelete = collectedBlocks.getToDeleteList();
+    for(BlockInfo blk : snapshotBlocks) {
       if(toDelete.contains(blk))
         collectedBlocks.removeDeleteBlock(blk);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 39180b1..f67d25a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -109,7 +109,7 @@ import com.google.common.annotations.VisibleForTesting;
 @InterfaceAudience.Private
 public class NamenodeFsck implements DataEncryptionKeyFactory {
   public static final Log LOG = LogFactory.getLog(NameNode.class.getName());
-  
+
   // return string marking fsck status
   public static final String CORRUPT_STATUS = "is CORRUPT";
   public static final String HEALTHY_STATUS = "is HEALTHY";
@@ -117,7 +117,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   public static final String DECOMMISSIONED_STATUS = "is DECOMMISSIONED";
   public static final String NONEXISTENT_STATUS = "does not exist";
   public static final String FAILURE_STATUS = "FAILED";
-  
+
   private final NameNode namenode;
   private final NetworkTopology networktopology;
   private final int totalDatanodes;
@@ -143,14 +143,14 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
    */
   private boolean internalError = false;
 
-  /** 
+  /**
    * True if the user specified the -move option.
    *
    * Whe this option is in effect, we will copy salvaged blocks into the lost
    * and found. */
   private boolean doMove = false;
 
-  /** 
+  /**
    * True if the user specified the -delete option.
    *
    * Whe this option is in effect, we will delete corrupted files.
@@ -183,7 +183,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
    * @param remoteAddress source address of the fsck request
    */
   NamenodeFsck(Configuration conf, NameNode namenode,
-      NetworkTopology networktopology, 
+      NetworkTopology networktopology,
       Map<String,String[]> pmap, PrintWriter out,
       int totalDatanodes, InetAddress remoteAddress) {
     this.conf = conf;
@@ -199,7 +199,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     this.staleInterval =
         conf.getLong(DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
           DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT);
-    
+
     for (Iterator<String> it = pmap.keySet().iterator(); it.hasNext();) {
       String key = it.next();
       if (key.equals("path")) { this.path = pmap.get("path")[0]; }
@@ -251,7 +251,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       }
       BlockCollection bc = bm.getBlockCollection(blockInfo);
       INode iNode = (INode) bc;
-      NumberReplicas numberReplicas= bm.countNodes(block);
+      NumberReplicas numberReplicas= bm.countNodes(blockInfo);
       out.println("Block Id: " + blockId);
       out.println("Block belongs to: "+iNode.getFullPathName());
       out.println("No. of Expected Replica: " +
@@ -350,7 +350,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
           listCorruptFileBlocks();
           return;
         }
-        
+
         if (this.showStoragePolcies) {
           storageTypeSummary = new StoragePolicySummary(
               namenode.getNamesystem().getBlockManager().getStoragePolicies());
@@ -380,7 +380,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
 
         // DFSck client scans for the string HEALTHY/CORRUPT to check the status
         // of file system and return appropriate code. Changing the output
-        // string might break testcases. Also note this must be the last line 
+        // string might break testcases. Also note this must be the last line
         // of the report.
         if (res.isHealthy()) {
           out.print("\n\nThe filesystem under path '" + path + "' " + HEALTHY_STATUS);
@@ -423,7 +423,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         + " CORRUPT files");
     out.println();
   }
-  
+
   @VisibleForTesting
   void check(String parent, HdfsFileStatus file, Result res) throws IOException {
     String path = file.getFullName(parent);
@@ -480,7 +480,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       throws IOException {
     long fileLen = file.getLen();
     LocatedBlocks blocks = null;
-    FSNamesystem fsn = namenode.getNamesystem();
+    final FSNamesystem fsn = namenode.getNamesystem();
     fsn.readLock();
     try {
       blocks = fsn.getBlockLocations(
@@ -539,8 +539,10 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       ExtendedBlock block = lBlk.getBlock();
       BlockManager bm = namenode.getNamesystem().getBlockManager();
 
+      final BlockInfo storedBlock = bm.getStoredBlock(
+          block.getLocalBlock());
       // count decommissionedReplicas / decommissioningReplicas
-      NumberReplicas numberReplicas = bm.countNodes(block.getLocalBlock());
+      NumberReplicas numberReplicas = bm.countNodes(storedBlock);
       int decommissionedReplicas = numberReplicas.decommissioned();;
       int decommissioningReplicas = numberReplicas.decommissioning();
       res.decommissionedReplicas +=  decommissionedReplicas;
@@ -608,7 +610,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
             out.println();
           out.print(path + ": ");
         }
-        out.println(" Replica placement policy is violated for " + 
+        out.println(" Replica placement policy is violated for " +
                     block + ". " + blockPlacementStatus.getErrorDescription());
       }
 
@@ -743,7 +745,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       return false;
     }
   }
-  
+
   private void copyBlocksToLostFound(String parent, HdfsFileStatus file,
         LocatedBlocks blocks) throws IOException {
     final DFSClient dfs = new DFSClient(NameNode.getAddress(conf), conf);
@@ -784,7 +786,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
           fos = dfs.create(target + "/" + chain, true);
           chain++;
         }
-        
+
         // copy the block. It's a pity it's not abstracted from DFSInputStream ...
         try {
           copyBlock(dfs, lblock, fos);
@@ -802,7 +804,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         LOG.warn("Fsck: there were errors copying the remains of the " +
           "corrupted file " + fullName + " to /lost+found");
       } else {
-        LOG.info("Fsck: copied the remains of the corrupted file " + 
+        LOG.info("Fsck: copied the remains of the corrupted file " +
           fullName + " to /lost+found");
       }
     } catch (Exception e) {
@@ -813,7 +815,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       dfs.close();
     }
   }
-      
+
   /*
    * XXX (ab) Bulk of this method is copied verbatim from {@link DFSClient}, which is
    * bad. Both places should be refactored to provide a method to copy blocks
@@ -824,12 +826,12 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     int failures = 0;
     InetSocketAddress targetAddr = null;
     TreeSet<DatanodeInfo> deadNodes = new TreeSet<DatanodeInfo>();
-    BlockReader blockReader = null; 
-    ExtendedBlock block = lblock.getBlock(); 
+    BlockReader blockReader = null;
+    ExtendedBlock block = lblock.getBlock();
 
     while (blockReader == null) {
       DatanodeInfo chosenNode;
-      
+
       try {
         chosenNode = bestNode(dfs, lblock.getLocations(), deadNodes);
         targetAddr = NetUtils.createSocketAddr(chosenNode.getXferAddr());
@@ -900,7 +902,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         bytesRead += cnt;
       }
       if ( bytesRead != block.getNumBytes() ) {
-        throw new IOException("Recorded block size is " + block.getNumBytes() + 
+        throw new IOException("Recorded block size is " + block.getNumBytes() +
                               ", but datanode returned " +bytesRead+" bytes");
       }
     } catch (Exception e) {
@@ -937,12 +939,12 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     } while (deadNodes.contains(chosenNode));
     return chosenNode;
   }
-  
+
   private void lostFoundInit(DFSClient dfs) {
     lfInited = true;
     try {
       String lfName = "/lost+found";
-      
+
       final HdfsFileStatus lfStatus = dfs.getFileInfo(lfName);
       if (lfStatus == null) { // not exists
         lfInitedOk = dfs.mkdirs(lfName, null, true);
@@ -997,21 +999,21 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
 
     final short replication;
     final int minReplication;
-    
+
     Result(Configuration conf) {
-      this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, 
+      this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
                                             DFSConfigKeys.DFS_REPLICATION_DEFAULT);
       this.minReplication = (short)conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY,
                                             DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT);
     }
-    
+
     /**
      * DFS is considered healthy if there are no missing blocks.
      */
     boolean isHealthy() {
       return ((missingIds.size() == 0) && (corruptBlocks == 0));
     }
-    
+
     /** Add a missing block name, plus its size. */
     void addMissing(String id, long size) {
       missingIds.add(id);
@@ -1030,7 +1032,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         return 0.0f;
       return (float) (totalReplicas) / (float) totalBlocks;
     }
-    
+
     @Override
     public String toString() {
       StringBuilder res = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
index 95fc06b..1428482 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 
 /** SafeMode related operations. */
 @InterfaceAudience.Private
@@ -49,5 +49,5 @@ public interface SafeMode {
   public void incrementSafeBlockCount(int replication);
 
   /** Decrement number of blocks that reached minimal replication. */
-  public void decrementSafeBlockCount(Block b);
+  public void decrementSafeBlockCount(BlockInfo b);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index 23e610f..148135b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -69,9 +69,10 @@ public class BlockManagerTestUtil {
     final BlockManager bm = namesystem.getBlockManager();
     namesystem.readLock();
     try {
+      final BlockInfo storedBlock = bm.getStoredBlock(b);
       return new int[]{getNumberOfRacks(bm, b),
-          bm.countNodes(b).liveReplicas(),
-          bm.neededReplications.contains(b) ? 1 : 0};
+          bm.countNodes(storedBlock).liveReplicas(),
+          bm.neededReplications.contains(storedBlock) ? 1 : 0};
     } finally {
       namesystem.readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 5a82b15..396dff3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -440,14 +440,14 @@ public class TestBlockManager {
     return blockInfo;
   }
 
-  private DatanodeStorageInfo[] scheduleSingleReplication(Block block) {
+  private DatanodeStorageInfo[] scheduleSingleReplication(BlockInfo block) {
     // list for priority 1
-    List<Block> list_p1 = new ArrayList<Block>();
+    List<BlockInfo> list_p1 = new ArrayList<>();
     list_p1.add(block);
 
     // list of lists for each priority
-    List<List<Block>> list_all = new ArrayList<List<Block>>();
-    list_all.add(new ArrayList<Block>()); // for priority 0
+    List<List<BlockInfo>> list_all = new ArrayList<>();
+    list_all.add(new ArrayList<BlockInfo>()); // for priority 0
     list_all.add(list_p1); // for priority 1
 
     assertEquals("Block not initially pending replication", 0,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
index c3726f2..1c3f075 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
@@ -166,10 +166,11 @@ public class TestNodeCount {
 
   /* threadsafe read of the replication counts for this block */
   NumberReplicas countNodes(Block block, FSNamesystem namesystem) {
+    BlockManager blockManager = namesystem.getBlockManager();
     namesystem.readLock();
     try {
       lastBlock = block;
-      lastNum = namesystem.getBlockManager().countNodes(block);
+      lastNum = blockManager.countNodes(blockManager.getStoredBlock(block));
       return lastNum;
     }
     finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
index a86b573..2d7bb44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
@@ -117,7 +117,8 @@ public class TestOverReplicatedBlocks {
 
           // corrupt one won't be chosen to be excess one
           // without 4910 the number of live replicas would be 0: block gets lost
-          assertEquals(1, bm.countNodes(block.getLocalBlock()).liveReplicas());
+          assertEquals(1, bm.countNodes(
+              bm.getStoredBlock(block.getLocalBlock())).liveReplicas());
         }
       } finally {
         namesystem.writeUnlock();
@@ -219,7 +220,7 @@ public class TestOverReplicatedBlocks {
       out.close();
       ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, p);
       assertEquals("Expected only one live replica for the block", 1, bm
-          .countNodes(block.getLocalBlock()).liveReplicas());
+          .countNodes(bm.getStoredBlock(block.getLocalBlock())).liveReplicas());
     } finally {
       cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
index 090c6e8..dae23d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
@@ -53,6 +53,12 @@ public class TestPendingReplication {
   private static final int DFS_REPLICATION_INTERVAL = 1;
   // Number of datanodes in the cluster
   private static final int DATANODE_COUNT = 5;
+
+  private BlockInfo genBlockInfo(long id, long length, long gs) {
+    return new BlockInfoContiguous(new Block(id, length, gs),
+        (short) DATANODE_COUNT);
+  }
+
   @Test
   public void testPendingReplication() {
     PendingReplicationBlocks pendingReplications;
@@ -63,7 +69,7 @@ public class TestPendingReplication {
     //
     DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(10);
     for (int i = 0; i < storages.length; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       DatanodeStorageInfo[] targets = new DatanodeStorageInfo[i];
       System.arraycopy(storages, 0, targets, 0, i);
       pendingReplications.increment(block,
@@ -76,7 +82,7 @@ public class TestPendingReplication {
     //
     // remove one item and reinsert it
     //
-    Block blk = new Block(8, 8, 0);
+    BlockInfo blk = genBlockInfo(8, 8, 0);
     pendingReplications.decrement(blk, storages[7].getDatanodeDescriptor()); // removes one replica
     assertEquals("pendingReplications.getNumReplicas ",
                  7, pendingReplications.getNumReplicas(blk));
@@ -96,7 +102,7 @@ public class TestPendingReplication {
     // are sane.
     //
     for (int i = 0; i < 10; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       int numReplicas = pendingReplications.getNumReplicas(block);
       assertTrue(numReplicas == i);
     }
@@ -115,7 +121,7 @@ public class TestPendingReplication {
     }
 
     for (int i = 10; i < 15; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       pendingReplications.increment(block,
           DatanodeStorageInfo.toDatanodeDescriptors(
               DFSTestUtil.createDatanodeStorageInfos(i)));
@@ -180,7 +186,7 @@ public class TestPendingReplication {
       block = new Block(1, 1, 0);
       blockInfo = new BlockInfoContiguous(block, (short) 3);
 
-      pendingReplications.increment(block,
+      pendingReplications.increment(blockInfo,
           DatanodeStorageInfo.toDatanodeDescriptors(
               DFSTestUtil.createDatanodeStorageInfos(1)));
       BlockCollection bc = Mockito.mock(BlockCollection.class);
@@ -195,7 +201,8 @@ public class TestPendingReplication {
       // Add a second block to pendingReplications that has no
       // corresponding entry in blocksmap
       block = new Block(2, 2, 0);
-      pendingReplications.increment(block,
+      blockInfo = new BlockInfoContiguous(block, (short) 3);
+      pendingReplications.increment(blockInfo,
           DatanodeStorageInfo.toDatanodeDescriptors(
               DFSTestUtil.createDatanodeStorageInfos(1)));
 
@@ -275,7 +282,7 @@ public class TestPendingReplication {
 
       assertEquals(1, blkManager.pendingReplications.size());
       INodeFile fileNode = fsn.getFSDirectory().getINode4Write(file).asFile();
-      Block[] blocks = fileNode.getBlocks();
+      BlockInfo[] blocks = fileNode.getBlocks();
       assertEquals(DATANODE_COUNT - 1,
           blkManager.pendingReplications.getNumReplicas(blocks[0]));
 
@@ -381,9 +388,9 @@ public class TestPendingReplication {
       BlockManagerTestUtil.computeAllPendingWork(bm);
       BlockManagerTestUtil.updateState(bm);
       assertEquals(bm.getPendingReplicationBlocksCount(), 1L);
-      assertEquals(bm.pendingReplications.getNumReplicas(block.getBlock()
-          .getLocalBlock()), 2);
-      
+      BlockInfo storedBlock = bm.getStoredBlock(block.getBlock().getLocalBlock());
+      assertEquals(bm.pendingReplications.getNumReplicas(storedBlock), 2);
+
       // 4. delete the file
       fs.delete(filePath, true);
       // retry at most 10 times, each time sleep for 1s. Note that 10s is much

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
index 728934d..1a32892 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
@@ -58,7 +58,9 @@ public class TestRBWBlockInvalidation {
   
   private static NumberReplicas countReplicas(final FSNamesystem namesystem,
       ExtendedBlock block) {
-    return namesystem.getBlockManager().countNodes(block.getLocalBlock());
+    final BlockManager blockManager = namesystem.getBlockManager();
+    return blockManager.countNodes(blockManager.getStoredBlock(
+        block.getLocalBlock()));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git 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
index 6e98538..2812957 100644
--- 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
@@ -830,7 +830,11 @@ public class TestReplicationPolicy {
     assertEquals(targets.length, 2);
     assertTrue(isOnSameRack(targets[0], dataNodes[2]));
   }
-  
+
+  private BlockInfo genBlockInfo(long id) {
+    return new BlockInfoContiguous(new Block(id), (short) 3);
+  }
+
   /**
    * Test for the high priority blocks are processed before the low priority
    * blocks.
@@ -849,16 +853,17 @@ public class TestReplicationPolicy {
           .getNamesystem().getBlockManager().neededReplications;
       for (int i = 0; i < 100; i++) {
         // Adding the blocks directly to normal priority
-        neededReplications.add(new Block(ThreadLocalRandom.current()
-            .nextLong()), 2, 0, 3);
+
+        neededReplications.add(genBlockInfo(ThreadLocalRandom.current().
+            nextLong()), 2, 0, 3);
       }
       // Lets wait for the replication interval, to start process normal
       // priority blocks
       Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
       
       // Adding the block directly to high priority list
-      neededReplications.add(new Block(ThreadLocalRandom.current().nextLong()),
-          1, 0, 3);
+      neededReplications.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 1, 0, 3);
 
       // Lets wait for the replication interval
       Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
@@ -881,30 +886,31 @@ public class TestReplicationPolicy {
 
     for (int i = 0; i < 5; i++) {
       // Adding QUEUE_HIGHEST_PRIORITY block
-      underReplicatedBlocks.add(new Block(ThreadLocalRandom.current()
-          .nextLong()), 1, 0, 3);
+      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 1, 0, 3);
 
       // Adding QUEUE_VERY_UNDER_REPLICATED block
-      underReplicatedBlocks.add(new Block(ThreadLocalRandom.current()
-          .nextLong()), 2, 0, 7);
+      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 2, 0, 7);
 
       // Adding QUEUE_REPLICAS_BADLY_DISTRIBUTED block
-      underReplicatedBlocks.add(new Block(ThreadLocalRandom.current()
-          .nextLong()), 6, 0, 6);
+      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 6, 0, 6);
 
       // Adding QUEUE_UNDER_REPLICATED block
-      underReplicatedBlocks.add(new Block(ThreadLocalRandom.current()
-          .nextLong()), 5, 0, 6);
+      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 5, 0, 6);
 
       // Adding QUEUE_WITH_CORRUPT_BLOCKS block
-      underReplicatedBlocks.add(new Block(ThreadLocalRandom.current()
-          .nextLong()), 0, 0, 3);
+      underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+          nextLong()), 0, 0, 3);
     }
 
     // Choose 6 blocks from UnderReplicatedBlocks. Then it should pick 5 blocks
     // from
     // QUEUE_HIGHEST_PRIORITY and 1 block from QUEUE_VERY_UNDER_REPLICATED.
-    List<List<Block>> chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(6);
+    List<List<BlockInfo>> chosenBlocks =
+        underReplicatedBlocks.chooseUnderReplicatedBlocks(6);
     assertTheChosenBlocks(chosenBlocks, 5, 1, 0, 0, 0);
 
     // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 4 blocks from
@@ -914,8 +920,8 @@ public class TestReplicationPolicy {
     assertTheChosenBlocks(chosenBlocks, 0, 4, 5, 1, 0);
 
     // Adding QUEUE_HIGHEST_PRIORITY
-    underReplicatedBlocks.add(new Block(ThreadLocalRandom.current().nextLong()),
-        1, 0, 3);
+    underReplicatedBlocks.add(genBlockInfo(ThreadLocalRandom.current().
+        nextLong()), 1, 0, 3);
 
     // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 1 block from
     // QUEUE_HIGHEST_PRIORITY, 4 blocks from QUEUE_REPLICAS_BADLY_DISTRIBUTED
@@ -933,7 +939,7 @@ public class TestReplicationPolicy {
   
   /** asserts the chosen blocks with expected priority blocks */
   private void assertTheChosenBlocks(
-      List<List<Block>> chosenBlocks, int firstPrioritySize,
+      List<List<BlockInfo>> chosenBlocks, int firstPrioritySize,
       int secondPrioritySize, int thirdPrioritySize, int fourthPrioritySize,
       int fifthPrioritySize) {
     assertEquals(
@@ -1107,9 +1113,9 @@ public class TestReplicationPolicy {
   public void testUpdateDoesNotCauseSkippedReplication() {
     UnderReplicatedBlocks underReplicatedBlocks = new UnderReplicatedBlocks();
 
-    Block block1 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block2 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block3 = new Block(ThreadLocalRandom.current().nextLong());
+    BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block3 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
     // Adding QUEUE_VERY_UNDER_REPLICATED block
     final int block1CurReplicas = 2;
@@ -1123,7 +1129,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block3, 2, 0, 6);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1156,8 +1162,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block2 = new Block(ThreadLocalRandom.current().nextLong());
+    BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1165,7 +1171,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1203,8 +1209,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block2 = new Block(ThreadLocalRandom.current().nextLong());
+    BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1212,7 +1218,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1266,8 +1272,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(ThreadLocalRandom.current().nextLong());
-    Block block2 = new Block(ThreadLocalRandom.current().nextLong());
+    BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong());
+    BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1275,7 +1281,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
index e87a043..de36e07 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
@@ -28,6 +28,10 @@ import static org.junit.Assert.fail;
 
 public class TestUnderReplicatedBlockQueues {
 
+  private BlockInfo genBlockInfo(long id) {
+    return new BlockInfoContiguous(new Block(id), (short) 3);
+  }
+
   /**
    * Test that adding blocks with different replication counts puts them
    * into different queues
@@ -36,11 +40,11 @@ public class TestUnderReplicatedBlockQueues {
   @Test
   public void testBlockPriorities() throws Throwable {
     UnderReplicatedBlocks queues = new UnderReplicatedBlocks();
-    Block block1 = new Block(1);
-    Block block2 = new Block(2);
-    Block block_very_under_replicated = new Block(3);
-    Block block_corrupt = new Block(4);
-    Block block_corrupt_repl_one = new Block(5);
+    BlockInfo block1 = genBlockInfo(1);
+    BlockInfo block2 = genBlockInfo(2);
+    BlockInfo block_very_under_replicated = genBlockInfo(3);
+    BlockInfo block_corrupt = genBlockInfo(4);
+    BlockInfo block_corrupt_repl_one = genBlockInfo(5);
 
     //add a block with a single entry
     assertAdded(queues, block1, 1, 0, 3);
@@ -82,7 +86,7 @@ public class TestUnderReplicatedBlockQueues {
   }
 
   private void assertAdded(UnderReplicatedBlocks queues,
-                           Block block,
+                           BlockInfo block,
                            int curReplicas,
                            int decomissionedReplicas,
                            int expectedReplicas) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
index 8f99afb..90eb7d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -81,6 +82,7 @@ public class TestReadOnlySharedStorage {
   private DatanodeInfo readOnlyDataNode;
   
   private Block block;
+  private BlockInfo storedBlock;
 
   private ExtendedBlock extendedBlock;
 
@@ -132,7 +134,8 @@ public class TestReadOnlySharedStorage {
     LocatedBlock locatedBlock = getLocatedBlock();
     extendedBlock = locatedBlock.getBlock();
     block = extendedBlock.getLocalBlock();
-    
+    storedBlock = blockManager.getStoredBlock(block);
+
     assertThat(locatedBlock.getLocations().length, is(1));
     normalDataNode = locatedBlock.getLocations()[0];
     readOnlyDataNode = datanodeManager.getDatanode(cluster.getDataNodes().get(RO_NODE_INDEX).getDatanodeId());
@@ -188,7 +191,7 @@ public class TestReadOnlySharedStorage {
   }
   
   private void validateNumberReplicas(int expectedReplicas) throws IOException {
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.liveReplicas(), is(expectedReplicas));
     assertThat(numberReplicas.excessReplicas(), is(0));
     assertThat(numberReplicas.corruptReplicas(), is(0));
@@ -230,7 +233,7 @@ public class TestReadOnlySharedStorage {
         cluster.getNameNode(), normalDataNode.getXferAddr());
     
     // The live replica count should now be zero (since the NORMAL replica is offline)
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.liveReplicas(), is(0));
     
     // The block should be reported as under-replicated
@@ -263,7 +266,7 @@ public class TestReadOnlySharedStorage {
     waitForLocations(1);
     
     // However, the corrupt READ_ONLY_SHARED replica should *not* affect the overall corrupt replicas count
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.corruptReplicas(), is(0));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5a4ccbd0/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
index 37abc5b..228a667 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.junit.Test;
@@ -260,7 +261,9 @@ public class TestProcessCorruptBlocks {
   }
 
   private static NumberReplicas countReplicas(final FSNamesystem namesystem, ExtendedBlock block) {
-    return namesystem.getBlockManager().countNodes(block.getLocalBlock());
+    final BlockManager blockManager = namesystem.getBlockManager();
+    return blockManager.countNodes(blockManager.getStoredBlock(
+        block.getLocalBlock()));
   }
 
   private void corruptBlock(MiniDFSCluster cluster, FileSystem fs, final Path fileName,


[20/50] [abbrv] hadoop git commit: MAPREDUCE-6396. TestPipeApplication fails by NullPointerException. Contributed by Brahma Reddy Battula.

Posted by zj...@apache.org.
MAPREDUCE-6396. TestPipeApplication fails by NullPointerException. Contributed by Brahma Reddy Battula.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0eb0d6e0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0eb0d6e0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0eb0d6e0

Branch: refs/heads/YARN-2928
Commit: 0eb0d6e072a6f483a650bd8df034bf789ef360a6
Parents: 9fd568b
Author: Akira Ajisaka <aa...@apache.org>
Authored: Mon Jun 15 15:28:31 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:06 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                            | 3 +++
 .../org/apache/hadoop/mapred/pipes/TestPipeApplication.java     | 5 +++--
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0eb0d6e0/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 3c2e8f9..5b66604 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -221,6 +221,9 @@ Trunk (Unreleased)
     MAPREDUCE-6343. JobConf.parseMaximumHeapSizeMB() fails to parse value 
     greater than 2GB expressed in bytes. (Hao Xia via kasha)
 
+    MAPREDUCE-6396. TestPipeApplication fails by NullPointerException.
+    (Brahma Reddy Battula via aajisaka)
+
   BREAKDOWN OF MAPREDUCE-2841 (NATIVE TASK) SUBTASKS
 
     MAPREDUCE-5985. native-task: Fix build on macosx. Contributed by

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0eb0d6e0/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java
index 64fdf41..22c5f41 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/pipes/TestPipeApplication.java
@@ -36,6 +36,7 @@ import java.util.Map.Entry;
 
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.io.BooleanWritable;
@@ -94,7 +95,7 @@ public class TestPipeApplication {
       CombineOutputCollector<IntWritable, Text> output = new CombineOutputCollector<IntWritable, Text>(
               new Counters.Counter(), new Progress());
       FileSystem fs = new RawLocalFileSystem();
-      fs.setConf(conf);
+      fs.initialize(FsConstants.LOCAL_FS_URI, conf);
       Writer<IntWritable, Text> wr = new Writer<IntWritable, Text>(conf, fs.create(
               new Path(workSpace + File.separator + "outfile")), IntWritable.class,
               Text.class, null, null, true);
@@ -176,7 +177,7 @@ public class TestPipeApplication {
       FakeCollector output = new FakeCollector(new Counters.Counter(),
               new Progress());
       FileSystem fs = new RawLocalFileSystem();
-      fs.setConf(conf);
+      fs.initialize(FsConstants.LOCAL_FS_URI, conf);
       Writer<IntWritable, Text> wr = new Writer<IntWritable, Text>(conf, fs.create(
               new Path(workSpace.getAbsolutePath() + File.separator + "outfile")),
               IntWritable.class, Text.class, null, null, true);


[48/50] [abbrv] hadoop git commit: HDFS-6249. Output AclEntry in PBImageXmlWriter. Contributed by surendra singh lilhore.

Posted by zj...@apache.org.
HDFS-6249. Output AclEntry in PBImageXmlWriter. Contributed by surendra singh lilhore.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/19551cf5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/19551cf5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/19551cf5

Branch: refs/heads/YARN-2928
Commit: 19551cf5f984ed4c5c63a27dd3619fdfd84c99c1
Parents: c181680
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Jun 17 17:41:10 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:02 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +++
 .../offlineImageViewer/PBImageXmlWriter.java    | 20 +++++++++++++---
 .../TestOfflineImageViewerForAcl.java           | 24 ++++++++++++++++++++
 3 files changed, 44 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/19551cf5/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 8f563de..2545bcf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -653,6 +653,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8589. Fix unused imports in BPServiceActor and BlockReportLeaseManager
     (cmccabe)
 
+    HDFS-6249. Output AclEntry in PBImageXmlWriter.
+    (surendra singh lilhore via aajisaka)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19551cf5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
index f3fe886..4415c5d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/PBImageXmlWriter.java
@@ -29,6 +29,7 @@ import java.util.Comparator;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.permission.AclEntry;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoExpirationProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CacheDirectiveInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CachePoolInfoProto;
@@ -41,6 +42,7 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FileSummary;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructionSection.FileUnderConstructionEntry;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.INodeSymlink;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeReferenceSection;
@@ -51,7 +53,7 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.SnapshotSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.StringTableSection;
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.util.LimitInputStream;
-
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 /**
@@ -188,7 +190,7 @@ public final class PBImageXmlWriter {
   private void dumpINodeDirectory(INodeDirectory d) {
     o("mtime", d.getModificationTime()).o("permission",
         dumpPermission(d.getPermission()));
-
+    dumpAcls(d.getAcl());
     if (d.hasDsQuota() && d.hasNsQuota()) {
       o("nsquota", d.getNsQuota()).o("dsquota", d.getDsQuota());
     }
@@ -242,7 +244,7 @@ public final class PBImageXmlWriter {
         .o("atime", f.getAccessTime())
         .o("perferredBlockSize", f.getPreferredBlockSize())
         .o("permission", dumpPermission(f.getPermission()));
-
+    dumpAcls(f.getAcl());
     if (f.getBlocksCount() > 0) {
       out.print("<blocks>");
       for (BlockProto b : f.getBlocksList()) {
@@ -263,6 +265,18 @@ public final class PBImageXmlWriter {
     }
   }
 
+  private void dumpAcls(AclFeatureProto aclFeatureProto) {
+    ImmutableList<AclEntry> aclEntryList = FSImageFormatPBINode.Loader
+        .loadAclEntries(aclFeatureProto, stringTable);
+    if (aclEntryList.size() > 0) {
+      out.print("<acls>");
+      for (AclEntry aclEntry : aclEntryList) {
+        o("acl", aclEntry.toString());
+      }
+      out.print("</acls>");
+    }
+  }
+
   private void dumpINodeSection(InputStream in) throws IOException {
     INodeSection s = INodeSection.parseDelimitedFrom(in);
     out.print("<INodeSection>");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/19551cf5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java
index 56963a6..7104ba1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java
@@ -17,14 +17,23 @@
  */
 package org.apache.hadoop.hdfs.tools.offlineImageViewer;
 
+import java.io.ByteArrayOutputStream;
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.io.PrintStream;
+import java.io.RandomAccessFile;
+import java.io.StringReader;
 import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.HashMap;
 
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.parsers.SAXParser;
+import javax.xml.parsers.SAXParserFactory;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -42,6 +51,9 @@ import org.apache.hadoop.net.NetUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
 
 import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS;
 import static org.apache.hadoop.fs.permission.AclEntryScope.DEFAULT;
@@ -202,4 +214,16 @@ public class TestOfflineImageViewerForAcl {
       viewer.close();
     }
   }
+
+  @Test
+  public void testPBImageXmlWriterForAcl() throws Exception{
+    ByteArrayOutputStream output = new ByteArrayOutputStream();
+    PrintStream o = new PrintStream(output);
+    PBImageXmlWriter v = new PBImageXmlWriter(new Configuration(), o);
+    v.visit(new RandomAccessFile(originalFsimage, "r"));
+    SAXParserFactory spf = SAXParserFactory.newInstance();
+    SAXParser parser = spf.newSAXParser();
+    final String xml = output.toString();
+    parser.parse(new InputSource(new StringReader(xml)), new DefaultHandler());
+  }
 }


[07/50] [abbrv] hadoop git commit: YARN-3794. TestRMEmbeddedElector fails because of ambiguous LOG reference. Contributed by Chengbing Liu.

Posted by zj...@apache.org.
YARN-3794. TestRMEmbeddedElector fails because of ambiguous LOG reference.
Contributed by Chengbing Liu.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1bddd3f0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1bddd3f0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1bddd3f0

Branch: refs/heads/YARN-2928
Commit: 1bddd3f0b5314e68548ebc51b0fd406b8ae8f787
Parents: 78cf309
Author: Devaraj K <de...@apache.org>
Authored: Fri Jun 12 13:42:49 2015 +0530
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:04 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                  | 3 +++
 .../yarn/server/resourcemanager/TestRMEmbeddedElector.java       | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1bddd3f0/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b692fa4..1776a89 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -621,6 +621,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-3778. Fix Yarn resourcemanger CLI usage. (Brahma Reddy Battula via xgong)
 
+    YARN-3794. TestRMEmbeddedElector fails because of ambiguous LOG reference.
+    (Chengbing Liu via devaraj)
+
 Release 2.7.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1bddd3f0/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
index 1b0bf7e..20b1c0e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMEmbeddedElector.java
@@ -102,9 +102,9 @@ public class TestRMEmbeddedElector extends ClientBaseWithFixes {
                 ServiceFailedException {
               try {
                 callbackCalled.set(true);
-                LOG.info("Callback called. Sleeping now");
+                TestRMEmbeddedElector.LOG.info("Callback called. Sleeping now");
                 Thread.sleep(delayMs);
-                LOG.info("Sleep done");
+                TestRMEmbeddedElector.LOG.info("Sleep done");
               } catch (InterruptedException e) {
                 e.printStackTrace();
               }


[31/50] [abbrv] hadoop git commit: HDFS-7164. Feature documentation for HDFS-6581. (Contributed by Arpit Agarwal)

Posted by zj...@apache.org.
HDFS-7164. Feature documentation for HDFS-6581. (Contributed by Arpit Agarwal)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b0b8fcb4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b0b8fcb4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b0b8fcb4

Branch: refs/heads/YARN-2928
Commit: b0b8fcb4056151a0228dca496b63bbaa807969c1
Parents: e0e7365
Author: Arpit Agarwal <ar...@apache.org>
Authored: Tue Jun 16 19:05:44 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:18:59 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../site/markdown/CentralizedCacheManagement.md |   2 +
 .../src/site/markdown/MemoryStorage.md          | 130 +++++++++++++++++++
 .../site/resources/images/LazyPersistWrites.png | Bin 0 -> 107161 bytes
 hadoop-project/src/site/site.xml                |   1 +
 5 files changed, 135 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b8fcb4/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index a36e047..07cd4a8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -638,6 +638,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-8606. Cleanup DFSOutputStream by removing unwanted changes
     from HDFS-8386. (Rakesh R via szetszwo)
 
+    HDFS-7164. Feature documentation for HDFS-6581. (Arpit Agarwal)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b8fcb4/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/CentralizedCacheManagement.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/CentralizedCacheManagement.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/CentralizedCacheManagement.md
index b4f08c8..72c125d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/CentralizedCacheManagement.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/CentralizedCacheManagement.md
@@ -233,6 +233,8 @@ Be sure to configure the following:
 
     This determines the maximum amount of memory a DataNode will use for caching. On Unix-like systems, the "locked-in-memory size" ulimit (`ulimit -l`) of the DataNode user also needs to be increased to match this parameter (see below section on [OS Limits](#OS_Limits)). When setting this value, please remember that you will need space in memory for other things as well, such as the DataNode and application JVM heaps and the operating system page cache.
 
+    This setting is shared with the [Lazy Persist Writes feature](./MemoryStorage.html). The Data Node will ensure that the combined memory used by Lazy Persist Writes and Centralized Cache Management does not exceed the amount configured in `dfs.datanode.max.locked.memory`.
+
 #### Optional
 
 The following properties are not required, but may be specified for tuning:

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b8fcb4/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/MemoryStorage.md
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/MemoryStorage.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/MemoryStorage.md
new file mode 100644
index 0000000..1ed4b5a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/MemoryStorage.md
@@ -0,0 +1,130 @@
+<!---
+  Licensed 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. See accompanying LICENSE file.
+-->
+
+Memory Storage Support in HDFS
+==============================
+
+* [Introduction](#Introduction)
+* [Administrator Configuration](#Administrator_Configuration)
+    * [Limit RAM used for replicas in Memory](#Limit_RAM_used_for_replicas_in_Memory)
+    * [Setup RAM Disks on Data Nodes](#Setup_RAM_Disks_on_Data_Nodes)
+        * [Choosing `tmpfs` \(vs `ramfs`\)](#Choosing_`tmpfs`_\(vs_`ramfs`\))
+        * [Mount RAM Disks](#Mount_RAM_Disks)
+        * [Tag `tmpfs` volume with the RAM\_DISK Storage Type](#Tag_`tmpfs`_volume_with_the_RAM\_DISK_Storage_Type)
+        * [Ensure Storage Policies are enabled](#Ensure_Storage_Policies_are_enabled)
+* [Application Usage](#Application_Usage)
+    * [Use the LAZY\_PERSIST Storage Policy](#Use_the_LAZY\_PERSIST_Storage_Policy)
+        * [Invoke `hdfs storagepolicies` command for directories](#Invoke_hdfs_storagepolicies_command_for_directories)
+        * [Call `setStoragePolicy` method for directories](#Call_`setStoragePolicy`_method_for_directories)
+        * [Pass `LAZY_PERSIST` `CreateFlag` for new files](#Pass_`LAZY_PERSIST`_`CreateFlag`_for_new_files)
+
+Introduction
+------------
+
+HDFS supports writing to off-heap memory managed by the Data Nodes. The Data Nodes will flush in-memory data to disk asynchronously thus removing expensive disk IO and checksum computations from the performance-sensitive IO path, hence we call such writes *Lazy Persist* writes. HDFS provides best-effort persistence guarantees for Lazy Persist Writes. Rare data loss is possible in the event of a node restart before replicas are persisted to disk. Applications can choose to use Lazy Persist Writes to trade off some durability guarantees in favor of reduced latency.
+
+This feature is available starting with Apache Hadoop 2.6.0 and was developed under Jira [HDFS-6581](https://issues.apache.org/jira/browse/HDFS-6581).
+
+![Lazy Persist Writes](images/LazyPersistWrites.png)
+
+The target use cases are applications that would benefit from writing relatively low amounts of data (from a few GB up to tens of GBs depending on available memory) with low latency. Memory storage is for applications that run within the cluster and collocated with HDFS Data Nodes. We have observed that the latency overhead from network replication negates the benefits of writing to memory.
+
+Applications that use Lazy Persist Writes will continue to work by falling back to DISK storage if memory is insufficient or unconfigured.
+
+Administrator Configuration
+---------------------------
+
+This section enumerates the administrative steps required before applications can start using the feature in a cluster.
+
+## Limit RAM used for replicas in Memory
+
+First decide the amount of memory to be dedicated for replicas stored in memory. Set `dfs.datanode.max.locked.memory` accordingly in `hdfs-site.xml`. This is the same setting used by the [Centralized Cache Management](./CentralizedCacheManagement.html) feature. The Data Node will ensure that the combined memory used by Lazy Persist Writes and Centralized Cache Management does not exceed the amount configured in `dfs.datanode.max.locked.memory`.
+
+E.g. To reserve 32 GB for in-memory replicas
+
+        <property>
+          <name>dfs.datanode.max.locked.memory</name>
+          <value>34359738368</value>
+        </property>
+
+This memory is not allocated by the Data Node on startup.
+
+On Unix-like systems, the "locked-in-memory size" ulimit (`ulimit -l`) of the Data Node user also needs to be increased to match this parameter (see the related section on [OS Limits](./CentralizedCacheManagement.html#OS_Limits)). When setting this value, please remember that you will need space in memory for other things as well, such as the Data Node and application JVM heaps and the operating system page cache. You will also need memory for YARN containers if there is a YARN Node Manager process running on the same node as the Data Node.
+
+## Setup RAM Disks on Data Nodes
+
+Initialize a RAM disk on each Data Node. The choice of RAM Disk allows better data persistence across Data Node process restarts. The following setup will work on most Linux distributions. Using RAM disks on other platforms is not currently supported.
+
+### Choosing `tmpfs` \(vs `ramfs`\)
+
+Linux supports using two kinds of RAM disks - `tmpfs` and `ramfs`. The size of `tmpfs` is limited by the Linux kernel while `ramfs` grows to fill all available system memory. There is a downside to `tmpfs` since its contents can be swapped to disk under memory pressure. However many performance-sensitive deployments run with swapping disabled so we do not expect this to be an issue in practice.
+
+HDFS currently supports using `tmpfs` partitions. Support for adding `ramfs` is in progress (See [HDFS-8584](https://issues.apache.org/jira/browse/HDFS-8584)).
+
+### Mount RAM Disks
+
+Mount the RAM Disk partition with the Unix `mount` command. E.g. to mount a 32 GB `tmpfs` partition under `/mnt/dn-tmpfs/`
+
+        sudo mount -t tmpfs -o size=32g tmpfs /mnt/dn-tmpfs/
+
+It is recommended you create an entry in the `/etc/fstab` so the RAM Disk is recreated automatically on node restarts. Another option is to use a sub-directory under `/dev/shm` which is a `tmpfs` mount available by default on most Linux distributions. Ensure that the size of the mount is greater than or equal to your `dfs.datanode.max.locked.memory` setting else override it in `/etc/fstab`. Using more than one `tmpfs` partition per Data Node for Lazy Persist Writes is not recommended.
+
+### Tag `tmpfs` volume with the RAM\_DISK Storage Type
+
+Tag the `tmpfs` directory with the RAM_DISK storage type via the `dfs.datanode.data.dir` configuration setting in `hdfs-site.xml`. E.g. On a Data Node with three hard disk volumes `/grid/0`, `/grid/1` and `/grid/2` and a `tmpfs` mount `/mnt/dn-tmpfs`, `dfs.datanode.data.dir` must be set as follows:
+
+        <property>
+          <name>dfs.datanode.data.dir</name>
+          <value>/grid/0,/grid/1,/grid/2,[RAM_DISK]/mnt/dn-tmpfs</value>
+        </property>
+
+This step is crucial. Without the RAM_DISK tag, HDFS will treat the `tmpfs` volume as non-volatile storage and data will not be saved to persistent storage. You will lose data on node restart.
+
+### Ensure Storage Policies are enabled
+
+Ensure that the global setting to turn on Storage Policies is enabled [as documented here](ArchivalStorage.html#Configuration). This setting is on by default.
+
+
+Application Usage
+-----------------
+
+## Use the LAZY\_PERSIST Storage Policy
+
+Applications indicate that HDFS can use Lazy Persist Writes for a file with the `LAZY_PERSIST` storage policy. Administrative privileges are *not* required to set the policy and it can be set in one of three ways.
+
+### Invoke `hdfs storagepolicies` command for directories
+
+Setting the policy on a directory causes it to take effect for all new files in the directory. The `hdfs storagepolicies` command can be used to set the policy as described in the [Storage Policies documentation](ArchivalStorage.html#Storage_Policy_Commands).
+
+        hdfs storagepolicies -setStoragePolicy -path <path> -policy LAZY_PERSIST
+
+### Call `setStoragePolicy` method for directories
+
+Starting with Apache Hadoop 2.8.0, an application can programmatically set the Storage Policy with `FileSystem.setStoragePolicy`. E.g.
+
+        fs.setStoragePolicy(path, "LAZY_PERSIST");
+
+### Pass `LAZY_PERSIST` `CreateFlag` for new files
+
+An application can pass `CreateFlag#LAZY_PERSIST` when creating a new file with `FileSystem#create` API. E.g.
+
+        FSDataOutputStream fos =
+            fs.create(
+                path,
+                FsPermission.getFileDefault(),
+                EnumSet.of(CreateFlag.CREATE, CreateFlag.LAZY_PERSIST),
+                bufferLength,
+                replicationFactor,
+                blockSize,
+                null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b8fcb4/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png b/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png
new file mode 100644
index 0000000..b2bde93
Binary files /dev/null and b/hadoop-hdfs-project/hadoop-hdfs/src/site/resources/images/LazyPersistWrites.png differ

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b0b8fcb4/hadoop-project/src/site/site.xml
----------------------------------------------------------------------
diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml
index 865d63e..55be0d9 100644
--- a/hadoop-project/src/site/site.xml
+++ b/hadoop-project/src/site/site.xml
@@ -96,6 +96,7 @@
       <item name="Transparent Encryption" href="hadoop-project-dist/hadoop-hdfs/TransparentEncryption.html"/>
       <item name="Multihoming" href="hadoop-project-dist/hadoop-hdfs/HdfsMultihoming.html"/>
       <item name="Storage Policies" href="hadoop-project-dist/hadoop-hdfs/ArchivalStorage.html"/>
+      <item name="Memory Storage Support" href="hadoop-project-dist/hadoop-hdfs/MemoryStorage.html"/>
       <item name="Synthetic Load Generator" href="hadoop-project-dist/hadoop-hdfs/SLGUserGuide.html"/>
     </menu>
 


[22/50] [abbrv] hadoop git commit: HDFS-8591. Remove support for deprecated configuration key dfs.namenode.decommission.nodes.per.interval.

Posted by zj...@apache.org.
HDFS-8591. Remove support for deprecated configuration key dfs.namenode.decommission.nodes.per.interval.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/090aa6ce
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/090aa6ce
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/090aa6ce

Branch: refs/heads/YARN-2928
Commit: 090aa6ce3a9f0c59834caba4a975ce5282beda5e
Parents: f28dc4f
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Jun 16 10:03:34 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:07 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../blockmanagement/DecommissionManager.java    | 42 +++++---------------
 .../apache/hadoop/hdfs/TestDecommission.java    | 26 ------------
 3 files changed, 12 insertions(+), 59 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/090aa6ce/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 0ae2882..e0ef52f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -31,6 +31,9 @@ Trunk (Unreleased)
  
     HDFS-8135. Remove the deprecated FSConstants class. (Li Lu via wheat9)
 
+    HDFS-8591. Remove support for deprecated configuration key
+    dfs.namenode.decommission.nodes.per.interval. (wang)
+
   NEW FEATURES
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/090aa6ce/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 639d2b0..797d031 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -136,29 +136,20 @@ public class DecommissionManager {
     checkArgument(intervalSecs >= 0, "Cannot set a negative " +
         "value for " + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY);
 
-    // By default, the new configuration key overrides the deprecated one.
-    // No # node limit is set.
     int blocksPerInterval = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY,
         DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_DEFAULT);
-    int nodesPerInterval = Integer.MAX_VALUE;
 
-    // If the expected key isn't present and the deprecated one is, 
-    // use the deprecated one into the new one. This overrides the 
-    // default.
-    //
-    // Also print a deprecation warning.
     final String deprecatedKey =
         "dfs.namenode.decommission.nodes.per.interval";
     final String strNodes = conf.get(deprecatedKey);
     if (strNodes != null) {
-      nodesPerInterval = Integer.parseInt(strNodes);
-      blocksPerInterval = Integer.MAX_VALUE;
-      LOG.warn("Using deprecated configuration key {} value of {}.",
-          deprecatedKey, nodesPerInterval); 
+      LOG.warn("Deprecated configuration key {} will be ignored.",
+          deprecatedKey);
       LOG.warn("Please update your configuration to use {} instead.", 
           DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY);
     }
+
     checkArgument(blocksPerInterval > 0,
         "Must set a positive value for "
         + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_BLOCKS_PER_INTERVAL_KEY);
@@ -170,15 +161,14 @@ public class DecommissionManager {
         "value for "
         + DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_MAX_CONCURRENT_TRACKED_NODES);
 
-    monitor = new Monitor(blocksPerInterval, 
-        nodesPerInterval, maxConcurrentTrackedNodes);
+    monitor = new Monitor(blocksPerInterval, maxConcurrentTrackedNodes);
     executor.scheduleAtFixedRate(monitor, intervalSecs, intervalSecs,
         TimeUnit.SECONDS);
 
     LOG.debug("Activating DecommissionManager with interval {} seconds, " +
-            "{} max blocks per interval, {} max nodes per interval, " +
+            "{} max blocks per interval, " +
             "{} max concurrently tracked nodes.", intervalSecs,
-        blocksPerInterval, nodesPerInterval, maxConcurrentTrackedNodes);
+        blocksPerInterval, maxConcurrentTrackedNodes);
   }
 
   /**
@@ -334,10 +324,6 @@ public class DecommissionManager {
      */
     private final int numBlocksPerCheck;
     /**
-     * The maximum number of nodes to check per tick.
-     */
-    private final int numNodesPerCheck;
-    /**
      * The maximum number of nodes to track in decomNodeBlocks. A value of 0
      * means no limit.
      */
@@ -348,7 +334,7 @@ public class DecommissionManager {
     private int numBlocksChecked = 0;
     /**
      * The number of nodes that have been checked on this tick. Used for 
-     * testing.
+     * statistics.
      */
     private int numNodesChecked = 0;
     /**
@@ -357,10 +343,8 @@ public class DecommissionManager {
     private DatanodeDescriptor iterkey = new DatanodeDescriptor(new 
         DatanodeID("", "", "", 0, 0, 0, 0));
 
-    Monitor(int numBlocksPerCheck, int numNodesPerCheck, int 
-        maxConcurrentTrackedNodes) {
+    Monitor(int numBlocksPerCheck, int maxConcurrentTrackedNodes) {
       this.numBlocksPerCheck = numBlocksPerCheck;
-      this.numNodesPerCheck = numNodesPerCheck;
       this.maxConcurrentTrackedNodes = maxConcurrentTrackedNodes;
     }
 
@@ -369,12 +353,6 @@ public class DecommissionManager {
       return numBlocksChecked >= numBlocksPerCheck;
     }
 
-    @Deprecated
-    private boolean exceededNumNodesPerCheck() {
-      LOG.trace("Processed {} nodes so far this tick", numNodesChecked);
-      return numNodesChecked >= numNodesPerCheck;
-    }
-
     @Override
     public void run() {
       if (!namesystem.isRunning()) {
@@ -416,9 +394,7 @@ public class DecommissionManager {
           it = new CyclicIteration<>(decomNodeBlocks, iterkey).iterator();
       final LinkedList<DatanodeDescriptor> toRemove = new LinkedList<>();
 
-      while (it.hasNext()
-          && !exceededNumBlocksPerCheck()
-          && !exceededNumNodesPerCheck()) {
+      while (it.hasNext() && !exceededNumBlocksPerCheck()) {
         numNodesChecked++;
         final Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>
             entry = it.next();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/090aa6ce/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
index 1f18014..8f965ad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
@@ -1045,32 +1045,6 @@ public class TestDecommission {
     doDecomCheck(datanodeManager, decomManager, 1);
   }
 
-  @Deprecated
-  @Test(timeout=120000)
-  public void testNodesPerInterval() throws Exception {
-    Configuration newConf = new Configuration(conf);
-    org.apache.log4j.Logger.getLogger(DecommissionManager.class)
-        .setLevel(Level.TRACE);
-    // Set the deprecated configuration key which limits the # of nodes per 
-    // interval
-    newConf.setInt("dfs.namenode.decommission.nodes.per.interval", 1);
-    // Disable the normal monitor runs
-    newConf.setInt(DFSConfigKeys.DFS_NAMENODE_DECOMMISSION_INTERVAL_KEY,
-        Integer.MAX_VALUE);
-    startCluster(1, 3, newConf);
-    final FileSystem fs = cluster.getFileSystem();
-    final DatanodeManager datanodeManager =
-        cluster.getNamesystem().getBlockManager().getDatanodeManager();
-    final DecommissionManager decomManager = datanodeManager.getDecomManager();
-
-    // Write a 3 block file, so each node has one block. Should scan 1 node 
-    // each time.
-    DFSTestUtil.createFile(fs, new Path("/file1"), 64, (short) 3, 0xBAD1DEA);
-    for (int i=0; i<3; i++) {
-      doDecomCheck(datanodeManager, decomManager, 1);
-    }
-  }
-
   private void doDecomCheck(DatanodeManager datanodeManager,
       DecommissionManager decomManager, int expectedNumCheckedNodes)
       throws IOException, ExecutionException, InterruptedException {


[06/50] [abbrv] hadoop git commit: HADOOP-11971. Move test utilities for tracing from hadoop-hdfs to hadoop-common. Contributed by Masatake Iwasaki.

Posted by zj...@apache.org.
HADOOP-11971. Move test utilities for tracing from hadoop-hdfs to hadoop-common. Contributed by Masatake Iwasaki.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9eb54e45
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9eb54e45
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9eb54e45

Branch: refs/heads/YARN-2928
Commit: 9eb54e45951d0fe35a4087a129c3c69d27c6c4c7
Parents: 1bddd3f
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jun 12 07:25:15 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:04 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   3 +
 .../apache/hadoop/tracing/SetSpanReceiver.java  | 109 +++++++++++++++++++
 .../org/apache/hadoop/tracing/TestTracing.java  |  94 ++--------------
 .../TestTracingShortCircuitLocalRead.java       |   4 +-
 4 files changed, 124 insertions(+), 86 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eb54e45/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 4df6112..268b1db 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -645,6 +645,9 @@ Release 2.8.0 - UNRELEASED
 
     HADOOP-12055. Deprecate usage of NativeIO#link. (Andrew Wang via cnauroth)
 
+    HADOOP-11971. Move test utilities for tracing from hadoop-hdfs to
+    hadoop-common. (Masatake Iwasaki via aajisaka)
+
   OPTIMIZATIONS
 
     HADOOP-11785. Reduce the number of listStatus operation in distcp

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eb54e45/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/SetSpanReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/SetSpanReceiver.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/SetSpanReceiver.java
new file mode 100644
index 0000000..e242b74
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tracing/SetSpanReceiver.java
@@ -0,0 +1,109 @@
+/**
+ * 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.tracing;
+
+import com.google.common.base.Supplier;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.htrace.Span;
+import org.apache.htrace.SpanReceiver;
+import org.apache.htrace.HTraceConfiguration;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeoutException;
+import org.junit.Assert;
+
+/**
+ * Span receiver that puts all spans into a single set.
+ * This is useful for testing.
+ * <p/>
+ * We're not using HTrace's POJOReceiver here so as that doesn't
+ * push all the metrics to a static place, and would make testing
+ * SpanReceiverHost harder.
+ */
+public class SetSpanReceiver implements SpanReceiver {
+
+  public SetSpanReceiver(HTraceConfiguration conf) {
+  }
+
+  public void receiveSpan(Span span) {
+    SetHolder.spans.put(span.getSpanId(), span);
+  }
+
+  public void close() {
+  }
+
+  public static void clear() {
+    SetHolder.spans.clear();
+  }
+
+  public static int size() {
+    return SetHolder.spans.size();
+  }
+
+  public static Collection<Span> getSpans() {
+    return SetHolder.spans.values();
+  }
+
+  public static Map<String, List<Span>> getMap() {
+    return SetHolder.getMap();
+  }
+
+  public static class SetHolder {
+    public static ConcurrentHashMap<Long, Span> spans =
+        new ConcurrentHashMap<Long, Span>();
+
+    public static Map<String, List<Span>> getMap() {
+      Map<String, List<Span>> map = new HashMap<String, List<Span>>();
+
+      for (Span s : spans.values()) {
+        List<Span> l = map.get(s.getDescription());
+        if (l == null) {
+          l = new LinkedList<Span>();
+          map.put(s.getDescription(), l);
+        }
+        l.add(s);
+      }
+      return map;
+    }
+  }
+
+  static void assertSpanNamesFound(final String[] expectedSpanNames) {
+    try {
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          Map<String, List<Span>> map = SetSpanReceiver.SetHolder.getMap();
+          for (String spanName : expectedSpanNames) {
+            if (!map.containsKey(spanName)) {
+              return false;
+            }
+          }
+          return true;
+        }
+      }, 100, 1000);
+    } catch (TimeoutException e) {
+      Assert.fail("timed out to get expected spans: " + e.getMessage());
+    } catch (InterruptedException e) {
+      Assert.fail("interrupted while waiting spans: " + e.getMessage());
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eb54e45/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
index 7fb8ac5..58b3659 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracing.java
@@ -25,11 +25,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.htrace.HTraceConfiguration;
 import org.apache.htrace.Sampler;
 import org.apache.htrace.Span;
-import org.apache.htrace.SpanReceiver;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
 import org.junit.After;
@@ -37,17 +34,10 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeoutException;
-
-import com.google.common.base.Supplier;
 
 public class TestTracing {
 
@@ -60,9 +50,9 @@ public class TestTracing {
     // write and read without tracing started
     String fileName = "testTracingDisabled.dat";
     writeTestFile(fileName);
-    Assert.assertTrue(SetSpanReceiver.SetHolder.size() == 0);
+    Assert.assertTrue(SetSpanReceiver.size() == 0);
     readTestFile(fileName);
-    Assert.assertTrue(SetSpanReceiver.SetHolder.size() == 0);
+    Assert.assertTrue(SetSpanReceiver.size() == 0);
 
     writeWithTracing();
     readWithTracing();
@@ -91,10 +81,10 @@ public class TestTracing {
       "org.apache.hadoop.hdfs.protocol.ClientProtocol.addBlock",
       "ClientNamenodeProtocol#addBlock"
     };
-    assertSpanNamesFound(expectedSpanNames);
+    SetSpanReceiver.assertSpanNamesFound(expectedSpanNames);
 
     // The trace should last about the same amount of time as the test
-    Map<String, List<Span>> map = SetSpanReceiver.SetHolder.getMap();
+    Map<String, List<Span>> map = SetSpanReceiver.getMap();
     Span s = map.get("testWriteTraceHooks").get(0);
     Assert.assertNotNull(s);
     long spanStart = s.getStartTimeMillis();
@@ -127,7 +117,7 @@ public class TestTracing {
            .get(0).getTimelineAnnotations()
            .get(0).getMessage());
 
-    SetSpanReceiver.SetHolder.spans.clear();
+    SetSpanReceiver.clear();
   }
 
   public void readWithTracing() throws Exception {
@@ -145,10 +135,10 @@ public class TestTracing {
       "ClientNamenodeProtocol#getBlockLocations",
       "OpReadBlockProto"
     };
-    assertSpanNamesFound(expectedSpanNames);
+    SetSpanReceiver.assertSpanNamesFound(expectedSpanNames);
 
     // The trace should last about the same amount of time as the test
-    Map<String, List<Span>> map = SetSpanReceiver.SetHolder.getMap();
+    Map<String, List<Span>> map = SetSpanReceiver.getMap();
     Span s = map.get("testReadTraceHooks").get(0);
     Assert.assertNotNull(s);
 
@@ -159,10 +149,10 @@ public class TestTracing {
 
     // There should only be one trace id as it should all be homed in the
     // top trace.
-    for (Span span : SetSpanReceiver.SetHolder.spans.values()) {
+    for (Span span : SetSpanReceiver.getSpans()) {
       Assert.assertEquals(ts.getSpan().getTraceId(), span.getTraceId());
     }
-    SetSpanReceiver.SetHolder.spans.clear();
+    SetSpanReceiver.clear();
   }
 
   private void writeTestFile(String testFileName) throws Exception {
@@ -211,7 +201,7 @@ public class TestTracing {
         .build();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
-    SetSpanReceiver.SetHolder.spans.clear();
+    SetSpanReceiver.clear();
   }
 
   @After
@@ -219,68 +209,4 @@ public class TestTracing {
     cluster.shutdown();
   }
 
-  static void assertSpanNamesFound(final String[] expectedSpanNames) {
-    try {
-      GenericTestUtils.waitFor(new Supplier<Boolean>() {
-        @Override
-        public Boolean get() {
-          Map<String, List<Span>> map = SetSpanReceiver.SetHolder.getMap();
-          for (String spanName : expectedSpanNames) {
-            if (!map.containsKey(spanName)) {
-              return false;
-            }
-          }
-          return true;
-        }
-      }, 100, 1000);
-    } catch (TimeoutException e) {
-      Assert.fail("timed out to get expected spans: " + e.getMessage());
-    } catch (InterruptedException e) {
-      Assert.fail("interrupted while waiting spans: " + e.getMessage());
-    }
-  }
-
-  /**
-   * Span receiver that puts all spans into a single set.
-   * This is useful for testing.
-   * <p/>
-   * We're not using HTrace's POJOReceiver here so as that doesn't
-   * push all the metrics to a static place, and would make testing
-   * SpanReceiverHost harder.
-   */
-  public static class SetSpanReceiver implements SpanReceiver {
-
-    public SetSpanReceiver(HTraceConfiguration conf) {
-    }
-
-    public void receiveSpan(Span span) {
-      SetHolder.spans.put(span.getSpanId(), span);
-    }
-
-    public void close() {
-    }
-
-    public static class SetHolder {
-      public static ConcurrentHashMap<Long, Span> spans = 
-          new ConcurrentHashMap<Long, Span>();
-          
-      public static int size() {
-        return spans.size();
-      }
-
-      public static Map<String, List<Span>> getMap() {
-        Map<String, List<Span>> map = new HashMap<String, List<Span>>();
-
-        for (Span s : spans.values()) {
-          List<Span> l = map.get(s.getDescription());
-          if (l == null) {
-            l = new LinkedList<Span>();
-            map.put(s.getDescription(), l);
-          }
-          l.add(s);
-        }
-        return map;
-      }
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9eb54e45/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
index 09ab350..0804a05 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tracing/TestTracingShortCircuitLocalRead.java
@@ -66,7 +66,7 @@ public class TestTracingShortCircuitLocalRead {
     conf = new Configuration();
     conf.set(DFSConfigKeys.DFS_CLIENT_HTRACE_PREFIX +
         SpanReceiverHost.SPAN_RECEIVERS_CONF_SUFFIX,
-        TestTracing.SetSpanReceiver.class.getName());
+        SetSpanReceiver.class.getName());
     conf.setLong("dfs.blocksize", 100 * 1024);
     conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.KEY, true);
     conf.setBoolean(HdfsClientConfigKeys.Read.ShortCircuit.SKIP_CHECKSUM_KEY, false);
@@ -92,7 +92,7 @@ public class TestTracingShortCircuitLocalRead {
         "OpRequestShortCircuitAccessProto",
         "ShortCircuitShmRequestProto"
       };
-      TestTracing.assertSpanNamesFound(expectedSpanNames);
+      SetSpanReceiver.assertSpanNamesFound(expectedSpanNames);
     } finally {
       dfs.close();
       cluster.shutdown();


[10/50] [abbrv] hadoop git commit: HADOOP-12001. Fixed LdapGroupsMapping to include configurable Posix UID and GID attributes during the search. Contributed by Patrick White.

Posted by zj...@apache.org.
HADOOP-12001. Fixed LdapGroupsMapping to include configurable Posix UID and GID attributes during the search. Contributed by Patrick White.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/68cc034c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/68cc034c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/68cc034c

Branch: refs/heads/YARN-2928
Commit: 68cc034c24683f24eb3fa6a9e0a2e4e9034555f3
Parents: e447ae3
Author: Vinod Kumar Vavilapalli <vi...@apache.org>
Authored: Mon Jun 15 14:22:34 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:05 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../hadoop/security/LdapGroupsMapping.java      | 29 +++++++++++++++-----
 .../src/main/resources/core-default.xml         | 18 ++++++++++++
 .../TestLdapGroupsMappingWithPosixGroup.java    | 25 +++++++++++------
 4 files changed, 60 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/68cc034c/hadoop-common-project/hadoop-common/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES.txt b/hadoop-common-project/hadoop-common/CHANGES.txt
index 268b1db..cdd396f 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -899,6 +899,9 @@ Release 2.7.1 - UNRELEASED
     HADOOP-12078. The default retry policy does not handle RetriableException
     correctly. (Arpit Agarwal)
 
+    HADOOP-12001. Fixed LdapGroupsMapping to include configurable Posix UID and
+    GID attributes during the search. (Patrick White via vinodkv)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68cc034c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
index df91b70..8f6203d 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/LdapGroupsMapping.java
@@ -150,10 +150,17 @@ public class LdapGroupsMapping
   public static final String GROUP_NAME_ATTR_DEFAULT = "cn";
 
   /*
+   * LDAP attribute names to use when doing posix-like lookups
+   */
+  public static final String POSIX_UID_ATTR_KEY = LDAP_CONFIG_PREFIX + ".posix.attr.uid.name";
+  public static final String POSIX_UID_ATTR_DEFAULT = "uidNumber";
+
+  public static final String POSIX_GID_ATTR_KEY = LDAP_CONFIG_PREFIX + ".posix.attr.gid.name";
+  public static final String POSIX_GID_ATTR_DEFAULT = "gidNumber";
+
+  /*
    * Posix attributes
    */
-  public static final String POSIX_UIDNUMBER = "uidNumber";
-  public static final String POSIX_GIDNUMBER = "gidNumber";
   public static final String POSIX_GROUP = "posixGroup";
   public static final String POSIX_ACCOUNT = "posixAccount";
 
@@ -186,6 +193,8 @@ public class LdapGroupsMapping
   private String userSearchFilter;
   private String groupMemberAttr;
   private String groupNameAttr;
+  private String posixUidAttr;
+  private String posixGidAttr;
   private boolean isPosix;
 
   public static int RECONNECT_RETRY_COUNT = 3;
@@ -256,8 +265,8 @@ public class LdapGroupsMapping
       if (isPosix) {
         String gidNumber = null;
         String uidNumber = null;
-        Attribute gidAttribute = result.getAttributes().get(POSIX_GIDNUMBER);
-        Attribute uidAttribute = result.getAttributes().get(POSIX_UIDNUMBER);
+        Attribute gidAttribute = result.getAttributes().get(posixGidAttr);
+        Attribute uidAttribute = result.getAttributes().get(posixUidAttr);
         if (gidAttribute != null) {
           gidNumber = gidAttribute.get().toString();
         }
@@ -267,7 +276,7 @@ public class LdapGroupsMapping
         if (uidNumber != null && gidNumber != null) {
           groupResults =
               ctx.search(baseDN,
-                  "(&"+ groupSearchFilter + "(|(" + POSIX_GIDNUMBER + "={0})" +
+                  "(&"+ groupSearchFilter + "(|(" + posixGidAttr + "={0})" +
                       "(" + groupMemberAttr + "={1})))",
                   new Object[] { gidNumber, uidNumber },
                   SEARCH_CONTROLS);
@@ -374,11 +383,17 @@ public class LdapGroupsMapping
         conf.get(GROUP_MEMBERSHIP_ATTR_KEY, GROUP_MEMBERSHIP_ATTR_DEFAULT);
     groupNameAttr =
         conf.get(GROUP_NAME_ATTR_KEY, GROUP_NAME_ATTR_DEFAULT);
+    posixUidAttr =
+        conf.get(POSIX_UID_ATTR_KEY, POSIX_UID_ATTR_DEFAULT);
+    posixGidAttr =
+        conf.get(POSIX_GID_ATTR_KEY, POSIX_GID_ATTR_DEFAULT);
 
     int dirSearchTimeout = conf.getInt(DIRECTORY_SEARCH_TIMEOUT, DIRECTORY_SEARCH_TIMEOUT_DEFAULT);
     SEARCH_CONTROLS.setTimeLimit(dirSearchTimeout);
-    // Limit the attributes returned to only those required to speed up the search. See HADOOP-10626 for more details.
-    SEARCH_CONTROLS.setReturningAttributes(new String[] {groupNameAttr});
+    // Limit the attributes returned to only those required to speed up the search.
+    // See HADOOP-10626 and HADOOP-12001 for more details.
+    SEARCH_CONTROLS.setReturningAttributes(
+        new String[] {groupNameAttr, posixUidAttr, posixGidAttr});
 
     this.conf = conf;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68cc034c/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
index ae23a75..13702ee 100644
--- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
+++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml
@@ -322,6 +322,24 @@ for ldap providers in the same way as above does.
 </property>
 
 <property>
+  <name>hadoop.security.group.mapping.ldap.posix.attr.uid.name</name>
+  <value>uidNumber</value>
+  <description>
+    The attribute of posixAccount to use when groups for membership.
+    Mostly useful for schemas wherein groups have memberUids that use an
+    attribute other than uidNumber.
+  </description>
+</property>
+
+<property>
+  <name>hadoop.security.group.mapping.ldap.posix.attr.gid.name</name>
+  <value>gidNumber</value>
+  <description>
+    The attribute of posixAccount indicating the group id.
+  </description>
+</property>
+
+<property>
   <name>hadoop.security.group.mapping.ldap.directory.search.timeout</name>
   <value>10000</value>
   <description>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/68cc034c/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
index 1d1a354..247f6c4 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestLdapGroupsMappingWithPosixGroup.java
@@ -52,14 +52,17 @@ public class TestLdapGroupsMappingWithPosixGroup
     SearchResult mockUserResult = mock(SearchResult.class);
     when(mockUserNamingEnum.nextElement()).thenReturn(mockUserResult);
 
+    Attribute mockUidNumberAttr = mock(Attribute.class);
+    Attribute mockGidNumberAttr = mock(Attribute.class);
     Attribute mockUidAttr = mock(Attribute.class);
-    Attribute mockGidAttr = mock(Attribute.class);
     Attributes mockAttrs = mock(Attributes.class);
 
-    when(mockUidAttr.get()).thenReturn("700");
-    when(mockGidAttr.get()).thenReturn("600");
-    when(mockAttrs.get(eq("uidNumber"))).thenReturn(mockUidAttr);
-    when(mockAttrs.get(eq("gidNumber"))).thenReturn(mockGidAttr);
+    when(mockUidAttr.get()).thenReturn("some_user");
+    when(mockUidNumberAttr.get()).thenReturn("700");
+    when(mockGidNumberAttr.get()).thenReturn("600");
+    when(mockAttrs.get(eq("uid"))).thenReturn(mockUidAttr);
+    when(mockAttrs.get(eq("uidNumber"))).thenReturn(mockUidNumberAttr);
+    when(mockAttrs.get(eq("gidNumber"))).thenReturn(mockGidNumberAttr);
 
     when(mockUserResult.getAttributes()).thenReturn(mockAttrs);
   }
@@ -85,6 +88,8 @@ public class TestLdapGroupsMappingWithPosixGroup
     conf.set(LdapGroupsMapping.USER_SEARCH_FILTER_KEY,
         "(objectClass=posixAccount)");
     conf.set(LdapGroupsMapping.GROUP_MEMBERSHIP_ATTR_KEY, "memberUid");
+    conf.set(LdapGroupsMapping.POSIX_UID_ATTR_KEY, "uidNumber");
+    conf.set(LdapGroupsMapping.POSIX_GID_ATTR_KEY, "gidNumber");
     conf.set(LdapGroupsMapping.GROUP_NAME_ATTR_KEY, "cn");
 
     mappingSpy.setConf(conf);
@@ -94,10 +99,14 @@ public class TestLdapGroupsMappingWithPosixGroup
 
     Assert.assertEquals(expectedGroups, groups);
 
+    mappingSpy.getConf().set(LdapGroupsMapping.POSIX_UID_ATTR_KEY, "uid");
+
+    Assert.assertEquals(expectedGroups, groups);
+
     // We should have searched for a user, and then two groups
     verify(mockContext, times(searchTimes)).search(anyString(),
-                                         anyString(),
-                                         any(Object[].class),
-                                         any(SearchControls.class));
+        anyString(),
+        any(Object[].class),
+        any(SearchControls.class));
   }
 }


[30/50] [abbrv] hadoop git commit: HDFS-4660. Block corruption can happen during pipeline recovery. Contributed by Kihwal Lee.

Posted by zj...@apache.org.
HDFS-4660. Block corruption can happen during pipeline recovery. Contributed by Kihwal Lee.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b5097681
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b5097681
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b5097681

Branch: refs/heads/YARN-2928
Commit: b5097681edbadec0f860a6bbcc6672a3c3169404
Parents: 5e962f6
Author: Kihwal Lee <ki...@apache.org>
Authored: Tue Jun 16 15:39:46 2015 -0500
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:18:58 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../hdfs/server/datanode/BlockReceiver.java     | 126 ++++++++++++++-----
 2 files changed, 96 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5097681/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 42588cc..b921f2c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1026,6 +1026,8 @@ Release 2.7.1 - UNRELEASED
 
     HDFS-8597. Fix TestFSImage#testZeroBlockSize on Windows. (Xiaoyu Yao)
 
+    HDFS-4660. Block corruption can happen during pipeline recovery (kihwal)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b5097681/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index c46892d..2468f43 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -588,29 +588,59 @@ class BlockReceiver implements Closeable {
       try {
         long onDiskLen = replicaInfo.getBytesOnDisk();
         if (onDiskLen<offsetInBlock) {
-          //finally write to the disk :
-          
-          if (onDiskLen % bytesPerChecksum != 0) { 
-            // prepare to overwrite last checksum
-            adjustCrcFilePosition();
+          // Normally the beginning of an incoming packet is aligned with the
+          // existing data on disk. If the beginning packet data offset is not
+          // checksum chunk aligned, the end of packet will not go beyond the
+          // next chunk boundary.
+          // When a failure-recovery is involved, the client state and the
+          // the datanode state may not exactly agree. I.e. the client may
+          // resend part of data that is already on disk. Correct number of
+          // bytes should be skipped when writing the data and checksum
+          // buffers out to disk.
+          long partialChunkSizeOnDisk = onDiskLen % bytesPerChecksum;
+          boolean alignedOnDisk = partialChunkSizeOnDisk == 0;
+          boolean alignedInPacket = firstByteInBlock % bytesPerChecksum == 0;
+
+          // Since data is always appended, not overwritten, partial CRC
+          // recalculation is necessary if the on-disk data is not chunk-
+          // aligned, regardless of whether the beginning of the data in
+          // the packet is chunk-aligned.
+          boolean doPartialCrc = !alignedOnDisk && !shouldNotWriteChecksum;
+
+          // If this is a partial chunk, then verify that this is the only
+          // chunk in the packet. If the starting offset is not chunk
+          // aligned, the packet should terminate at or before the next
+          // chunk boundary.
+          if (!alignedInPacket && len > bytesPerChecksum) {
+            throw new IOException("Unexpected packet data length for "
+                +  block + " from " + inAddr + ": a partial chunk must be "
+                + " sent in an individual packet (data length = " + len
+                +  " > bytesPerChecksum = " + bytesPerChecksum + ")");
           }
-          
-          // If this is a partial chunk, then read in pre-existing checksum
+
+          // If the last portion of the block file is not a full chunk,
+          // then read in pre-existing partial data chunk and recalculate
+          // the checksum so that the checksum calculation can continue
+          // from the right state.
           Checksum partialCrc = null;
-          if (!shouldNotWriteChecksum && firstByteInBlock % bytesPerChecksum != 0) {
+          if (doPartialCrc) {
             if (LOG.isDebugEnabled()) {
               LOG.debug("receivePacket for " + block 
-                  + ": bytesPerChecksum=" + bytesPerChecksum                  
-                  + " does not divide firstByteInBlock=" + firstByteInBlock);
+                  + ": previous write did not end at the chunk boundary."
+                  + " onDiskLen=" + onDiskLen);
             }
             long offsetInChecksum = BlockMetadataHeader.getHeaderSize() +
                 onDiskLen / bytesPerChecksum * checksumSize;
             partialCrc = computePartialChunkCrc(onDiskLen, offsetInChecksum);
           }
 
+          // The data buffer position where write will begin. If the packet
+          // data and on-disk data have no overlap, this will not be at the
+          // beginning of the buffer.
           int startByteToDisk = (int)(onDiskLen-firstByteInBlock) 
               + dataBuf.arrayOffset() + dataBuf.position();
 
+          // Actual number of data bytes to write.
           int numBytesToDisk = (int)(offsetInBlock-onDiskLen);
           
           // Write data to disk.
@@ -625,31 +655,63 @@ class BlockReceiver implements Closeable {
           final byte[] lastCrc;
           if (shouldNotWriteChecksum) {
             lastCrc = null;
-          } else if (partialCrc != null) {
-            // If this is a partial chunk, then verify that this is the only
-            // chunk in the packet. Calculate new crc for this chunk.
-            if (len > bytesPerChecksum) {
-              throw new IOException("Unexpected packet data length for "
-                  +  block + " from " + inAddr + ": a partial chunk must be "
-                  + " sent in an individual packet (data length = " + len
-                  +  " > bytesPerChecksum = " + bytesPerChecksum + ")");
+          } else {
+            int skip = 0;
+            byte[] crcBytes = null;
+
+            // First, overwrite the partial crc at the end, if necessary.
+            if (doPartialCrc) { // not chunk-aligned on disk
+              // Calculate new crc for this chunk.
+              int bytesToReadForRecalc =
+                  (int)(bytesPerChecksum - partialChunkSizeOnDisk);
+              if (numBytesToDisk < bytesToReadForRecalc) {
+                bytesToReadForRecalc = numBytesToDisk;
+              }
+
+              partialCrc.update(dataBuf.array(), startByteToDisk,
+                  bytesToReadForRecalc);
+              byte[] buf = FSOutputSummer.convertToByteStream(partialCrc,
+                  checksumSize);
+              crcBytes = copyLastChunkChecksum(buf, checksumSize, buf.length);
+              // prepare to overwrite last checksum
+              adjustCrcFilePosition();
+              checksumOut.write(buf);
+              if(LOG.isDebugEnabled()) {
+                LOG.debug("Writing out partial crc for data len " + len +
+                    ", skip=" + skip);
+              }
+              skip++; //  For the partial chunk that was just read.
             }
-            partialCrc.update(dataBuf.array(), startByteToDisk, numBytesToDisk);
-            byte[] buf = FSOutputSummer.convertToByteStream(partialCrc, checksumSize);
-            lastCrc = copyLastChunkChecksum(buf, checksumSize, buf.length);
-            checksumOut.write(buf);
-            if(LOG.isDebugEnabled()) {
-              LOG.debug("Writing out partial crc for data len " + len);
+
+            // Determine how many checksums need to be skipped up to the last
+            // boundary. The checksum after the boundary was already counted
+            // above. Only count the number of checksums skipped up to the
+            // boundary here.
+            long lastChunkBoundary = onDiskLen - (onDiskLen%bytesPerChecksum);
+            long skippedDataBytes = lastChunkBoundary - firstByteInBlock;
+
+            if (skippedDataBytes > 0) {
+              skip += (int)(skippedDataBytes / bytesPerChecksum) +
+                  ((skippedDataBytes % bytesPerChecksum == 0) ? 0 : 1);
             }
-            partialCrc = null;
-          } else {
-            // write checksum
+            skip *= checksumSize; // Convert to number of bytes
+
+            // write the rest of checksum
             final int offset = checksumBuf.arrayOffset() +
-                checksumBuf.position();
-            final int end = offset + checksumLen;
-            lastCrc = copyLastChunkChecksum(checksumBuf.array(), checksumSize,
-                end);
-            checksumOut.write(checksumBuf.array(), offset, checksumLen);
+                checksumBuf.position() + skip;
+            final int end = offset + checksumLen - skip;
+            // If offset > end, there is no more checksum to write.
+            // I.e. a partial chunk checksum rewrite happened and there is no
+            // more to write after that.
+            if (offset > end) {
+              assert crcBytes != null;
+              lastCrc = crcBytes;
+            } else {
+              final int remainingBytes = checksumLen - skip;
+              lastCrc = copyLastChunkChecksum(checksumBuf.array(),
+                  checksumSize, end);
+              checksumOut.write(checksumBuf.array(), offset, remainingBytes);
+            }
           }
 
           /// flush entire packet, sync if requested


[27/50] [abbrv] hadoop git commit: HDFS-8597. Fix TestFSImage#testZeroBlockSize on Windows. Contributed by Xiaoyu Yao.

Posted by zj...@apache.org.
HDFS-8597. Fix TestFSImage#testZeroBlockSize on Windows. Contributed by Xiaoyu Yao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/12b46e18
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/12b46e18
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/12b46e18

Branch: refs/heads/YARN-2928
Commit: 12b46e18e0094c46ca60e7f0ec4f333d4029a96c
Parents: 090aa6c
Author: Xiaoyu Yao <xy...@apache.org>
Authored: Tue Jun 16 12:38:07 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:18:58 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 2 ++
 .../org/apache/hadoop/hdfs/server/datanode/StorageLocation.java  | 4 ++--
 2 files changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/12b46e18/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e0ef52f..42588cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1024,6 +1024,8 @@ Release 2.7.1 - UNRELEASED
     HDFS-8576.  Lease recovery should return true if the lease can be released
     and the file can be closed.  (J.Andreina via szetszwo)
 
+    HDFS-8597. Fix TestFSImage#testZeroBlockSize on Windows. (Xiaoyu Yao)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/12b46e18/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
index 126086f..5c8dd85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java
@@ -26,8 +26,8 @@ import java.net.URI;
 import java.util.regex.Matcher;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.util.StringUtils;
 
 /**
@@ -94,7 +94,7 @@ public class StorageLocation {
       }
     }
 
-    return new StorageLocation(storageType, Util.stringAsURI(location));
+    return new StorageLocation(storageType, new Path(location).toUri());
   }
 
   @Override


[15/50] [abbrv] hadoop git commit: HDFS-8596. TestDistributedFileSystem et al tests are broken in branch-2 due to incorrect setting of "datanode" attribute. Contributed by Yongjun Zhang.

Posted by zj...@apache.org.
HDFS-8596. TestDistributedFileSystem et al tests are broken in branch-2 due to incorrect setting of "datanode" attribute. Contributed by Yongjun Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2b93ab3d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2b93ab3d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2b93ab3d

Branch: refs/heads/YARN-2928
Commit: 2b93ab3d26ce275e5f35bfc7d116abb4588d9249
Parents: fb7f8ec
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Sun Jun 14 11:20:32 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:05 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                       | 3 +++
 .../java/org/apache/hadoop/hdfs/server/datanode/DataNode.java     | 2 +-
 .../hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java       | 3 ++-
 3 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b93ab3d/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 9aabd34..1caa8c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1002,6 +1002,9 @@ Release 2.7.1 - UNRELEASED
     HDFS-8572. DN always uses HTTP/localhost@REALM principals in SPNEGO.
     (wheat9)
 
+    HDFS-8596. TestDistributedFileSystem et al tests are broken in branch-2
+    due to incorrect setting of "datanode" attribute. (Yongjun Zhang)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b93ab3d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index ed2925b..3bd131e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -762,7 +762,7 @@ public class DataNode extends ReconfigurableBase
     ServerSocketChannel httpServerChannel = secureResources != null ?
         secureResources.getHttpServerChannel() : null;
 
-    this.httpServer = new DatanodeHttpServer(conf, httpServerChannel);
+    this.httpServer = new DatanodeHttpServer(conf, this, httpServerChannel);
     httpServer.start();
     if (httpServer.getHttpAddress() != null) {
       infoPort = httpServer.getHttpAddress().getPort();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2b93ab3d/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
index e9ad92f..62c98e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java
@@ -75,6 +75,7 @@ public class DatanodeHttpServer implements Closeable {
   static final Log LOG = LogFactory.getLog(DatanodeHttpServer.class);
 
   public DatanodeHttpServer(final Configuration conf,
+      final DataNode datanode,
       final ServerSocketChannel externalHttpChannel)
     throws IOException {
     this.conf = conf;
@@ -91,7 +92,7 @@ public class DatanodeHttpServer implements Closeable {
 
     this.infoServer = builder.build();
 
-    this.infoServer.setAttribute("datanode", this);
+    this.infoServer.setAttribute("datanode", datanode);
     this.infoServer.setAttribute(JspHelper.CURRENT_CONF, conf);
     this.infoServer.addServlet(null, "/blockScannerReport",
                                BlockScanner.Servlet.class);


[50/50] [abbrv] hadoop git commit: HDFS-8605. Merge Refactor of DFSOutputStream from HDFS-7285 branch. (vinayakumarb)

Posted by zj...@apache.org.
HDFS-8605. Merge Refactor of DFSOutputStream from HDFS-7285 branch. (vinayakumarb)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8c036a14
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8c036a14
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8c036a14

Branch: refs/heads/YARN-2928
Commit: 8c036a14e3b53ea398d154b0ab4d7eb7b43d59f1
Parents: d6e632a
Author: Andrew Wang <wa...@apache.org>
Authored: Thu Jun 18 08:48:09 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:03 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../org/apache/hadoop/hdfs/DFSOutputStream.java | 59 ++++++++++----------
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  7 ++-
 3 files changed, 40 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c036a14/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 2545bcf..a61cf78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -656,6 +656,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-6249. Output AclEntry in PBImageXmlWriter.
     (surendra singh lilhore via aajisaka)
 
+    HDFS-8605. Merge Refactor of DFSOutputStream from HDFS-7285 branch.
+    (vinayakumarb via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c036a14/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 4622be6..c16aef2 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -64,6 +64,8 @@ import org.apache.hadoop.util.Time;
 import org.apache.htrace.Sampler;
 import org.apache.htrace.Trace;
 import org.apache.htrace.TraceScope;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -86,6 +88,7 @@ import com.google.common.base.Preconditions;
 @InterfaceAudience.Private
 public class DFSOutputStream extends FSOutputSummer
     implements Syncable, CanSetDropBehind {
+  static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class);
   /**
    * Number of times to retry creating a file when there are transient 
    * errors (typically related to encryption zones and KeyProvider operations).
@@ -413,21 +416,30 @@ public class DFSOutputStream extends FSOutputSummer
     //
     if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
         getStreamer().getBytesCurBlock() == blockSize) {
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("DFSClient writeChunk packet full seqno=" +
-            currentPacket.getSeqno() +
-            ", src=" + src +
-            ", bytesCurBlock=" + getStreamer().getBytesCurBlock() +
-            ", blockSize=" + blockSize +
-            ", appendChunk=" + getStreamer().getAppendChunk());
-      }
-      getStreamer().waitAndQueuePacket(currentPacket);
-      currentPacket = null;
+      enqueueCurrentPacketFull();
+    }
+  }
 
-      adjustChunkBoundary();
+  void enqueueCurrentPacket() throws IOException {
+    getStreamer().waitAndQueuePacket(currentPacket);
+    currentPacket = null;
+  }
 
-      endBlock();
-    }
+  void enqueueCurrentPacketFull() throws IOException {
+    LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
+        + " appendChunk={}, {}", currentPacket, src, getStreamer()
+        .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
+        getStreamer());
+    enqueueCurrentPacket();
+    adjustChunkBoundary();
+    endBlock();
+  }
+
+  /** create an empty packet to mark the end of the block. */
+  void setCurrentPacketToEmpty() throws InterruptedIOException {
+    currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
+        getStreamer().getAndIncCurrentSeqno(), true);
+    currentPacket.setSyncBlock(shouldSyncBlock);
   }
 
   /**
@@ -457,11 +469,8 @@ public class DFSOutputStream extends FSOutputSummer
    */
   protected void endBlock() throws IOException {
     if (getStreamer().getBytesCurBlock() == blockSize) {
-      currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
-          getStreamer().getAndIncCurrentSeqno(), true);
-      currentPacket.setSyncBlock(shouldSyncBlock);
-      getStreamer().waitAndQueuePacket(currentPacket);
-      currentPacket = null;
+      setCurrentPacketToEmpty();
+      enqueueCurrentPacket();
       getStreamer().setBytesCurBlock(0);
       lastFlushOffset = 0;
     }
@@ -586,8 +595,7 @@ public class DFSOutputStream extends FSOutputSummer
         }
         if (currentPacket != null) {
           currentPacket.setSyncBlock(isSync);
-          getStreamer().waitAndQueuePacket(currentPacket);
-          currentPacket = null;
+          enqueueCurrentPacket();
         }
         if (endBlock && getStreamer().getBytesCurBlock() > 0) {
           // Need to end the current block, thus send an empty packet to
@@ -595,8 +603,7 @@ public class DFSOutputStream extends FSOutputSummer
           currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
               getStreamer().getAndIncCurrentSeqno(), true);
           currentPacket.setSyncBlock(shouldSyncBlock || isSync);
-          getStreamer().waitAndQueuePacket(currentPacket);
-          currentPacket = null;
+          enqueueCurrentPacket();
           getStreamer().setBytesCurBlock(0);
           lastFlushOffset = 0;
         } else {
@@ -775,15 +782,11 @@ public class DFSOutputStream extends FSOutputSummer
       flushBuffer();       // flush from all upper layers
 
       if (currentPacket != null) {
-        getStreamer().waitAndQueuePacket(currentPacket);
-        currentPacket = null;
+        enqueueCurrentPacket();
       }
 
       if (getStreamer().getBytesCurBlock() != 0) {
-        // send an empty packet to mark the end of the block
-        currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
-            getStreamer().getAndIncCurrentSeqno(), true);
-        currentPacket.setSyncBlock(shouldSyncBlock);
+        setCurrentPacketToEmpty();
       }
 
       flushInternal();             // flush all data to Datanodes

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8c036a14/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index cecd5a0..8dd85b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -44,7 +44,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
 import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
@@ -1901,4 +1900,10 @@ class DataStreamer extends Daemon {
       s.close();
     }
   }
+
+  @Override
+  public String toString() {
+    return  (block == null? null: block.getLocalBlock())
+        + "@" + Arrays.toString(getNodes());
+  }
 }


[03/50] [abbrv] hadoop git commit: Add HDFS-7923 to CHANGES.txt

Posted by zj...@apache.org.
Add HDFS-7923 to CHANGES.txt


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/98591bd7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/98591bd7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/98591bd7

Branch: refs/heads/YARN-2928
Commit: 98591bd7ab33e081ea09cd0c6012495fd3ceb1f6
Parents: f746001
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Fri Jun 12 11:28:18 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:04 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/98591bd7/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index b388f69..e315db6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -623,6 +623,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8573. Move creation of restartMeta file logic from BlockReceiver to
     ReplicaInPipeline. (Eddy Xu via wang)
 
+    HDFS-7923. The DataNodes should rate-limit their full block reports by
+    asking the NN on heartbeat messages (cmccabe)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than


[38/50] [abbrv] hadoop git commit: HDFS-8238. Move ClientProtocol to the hdfs-client. Contributed by Takanobu Asanuma.

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb2ccade/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
deleted file mode 100644
index 55faf16..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ /dev/null
@@ -1,1459 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.EnumSet;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.crypto.CryptoProtocolVersion;
-import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries;
-import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FsServerDefaults;
-import org.apache.hadoop.fs.InvalidPathException;
-import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.XAttr;
-import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.inotify.EventBatchList;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.retry.AtMostOnce;
-import org.apache.hadoop.io.retry.Idempotent;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.KerberosInfo;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenInfo;
-
-/**********************************************************************
- * ClientProtocol is used by user code via 
- * {@link org.apache.hadoop.hdfs.DistributedFileSystem} class to communicate 
- * with the NameNode.  User code can manipulate the directory namespace, 
- * as well as open/close file streams, etc.
- *
- **********************************************************************/
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-@KerberosInfo(
-    serverPrincipal = DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY)
-@TokenInfo(DelegationTokenSelector.class)
-public interface ClientProtocol {
-
-  /**
-   * Until version 69, this class ClientProtocol served as both
-   * the client interface to the NN AND the RPC protocol used to 
-   * communicate with the NN.
-   * 
-   * This class is used by both the DFSClient and the 
-   * NN server side to insulate from the protocol serialization.
-   * 
-   * If you are adding/changing this interface then you need to 
-   * change both this class and ALSO related protocol buffer
-   * wire protocol definition in ClientNamenodeProtocol.proto.
-   * 
-   * For more details on protocol buffer wire protocol, please see 
-   * .../org/apache/hadoop/hdfs/protocolPB/overview.html
-   * 
-   * The log of historical changes can be retrieved from the svn).
-   * 69: Eliminate overloaded method names.
-   * 
-   * 69L is the last version id when this class was used for protocols
-   *  serialization. DO not update this version any further. 
-   */
-  public static final long versionID = 69L;
-  
-  ///////////////////////////////////////
-  // File contents
-  ///////////////////////////////////////
-  /**
-   * Get locations of the blocks of the specified file within the specified range.
-   * DataNode locations for each block are sorted by
-   * the proximity to the client.
-   * <p>
-   * Return {@link LocatedBlocks} which contains
-   * file length, blocks and their locations.
-   * DataNode locations for each block are sorted by
-   * the distance to the client's address.
-   * <p>
-   * The client will then have to contact 
-   * one of the indicated DataNodes to obtain the actual data.
-   * 
-   * @param src file name
-   * @param offset range start offset
-   * @param length range length
-   *
-   * @return file length and array of blocks with their locations
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> does not exist
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public LocatedBlocks getBlockLocations(String src,
-                                         long offset,
-                                         long length) 
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * Get server default values for a number of configuration params.
-   * @return a set of server default configuration values
-   * @throws IOException
-   */
-  @Idempotent
-  public FsServerDefaults getServerDefaults() throws IOException;
-
-  /**
-   * Create a new file entry in the namespace.
-   * <p>
-   * This will create an empty file specified by the source path.
-   * The path should reflect a full path originated at the root.
-   * The name-node does not have a notion of "current" directory for a client.
-   * <p>
-   * Once created, the file is visible and available for read to other clients.
-   * Although, other clients cannot {@link #delete(String, boolean)}, re-create or 
-   * {@link #rename(String, String)} it until the file is completed
-   * or explicitly as a result of lease expiration.
-   * <p>
-   * Blocks have a maximum size.  Clients that intend to create
-   * multi-block files must also use 
-   * {@link #addBlock}
-   *
-   * @param src path of the file being created.
-   * @param masked masked permission.
-   * @param clientName name of the current client.
-   * @param flag indicates whether the file should be 
-   * overwritten if it already exists or create if it does not exist or append.
-   * @param createParent create missing parent directory if true
-   * @param replication block replication factor.
-   * @param blockSize maximum block size.
-   * @param supportedVersions CryptoProtocolVersions supported by the client
-   * 
-   * @return the status of the created file, it could be null if the server
-   *           doesn't support returning the file status
-   * @throws AccessControlException If access is denied
-   * @throws AlreadyBeingCreatedException if the path does not exist.
-   * @throws DSQuotaExceededException If file creation violates disk space 
-   *           quota restriction
-   * @throws FileAlreadyExistsException If file <code>src</code> already exists
-   * @throws FileNotFoundException If parent of <code>src</code> does not exist
-   *           and <code>createParent</code> is false
-   * @throws ParentNotDirectoryException If parent of <code>src</code> is not a
-   *           directory.
-   * @throws NSQuotaExceededException If file creation violates name space 
-   *           quota restriction
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   *
-   * RuntimeExceptions:
-   * @throws InvalidPathException Path <code>src</code> is invalid
-   * <p>
-   * <em>Note that create with {@link CreateFlag#OVERWRITE} is idempotent.</em>
-   */
-  @AtMostOnce
-  public HdfsFileStatus create(String src, FsPermission masked,
-      String clientName, EnumSetWritable<CreateFlag> flag,
-      boolean createParent, short replication, long blockSize, 
-      CryptoProtocolVersion[] supportedVersions)
-      throws AccessControlException, AlreadyBeingCreatedException,
-      DSQuotaExceededException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
-
-  /**
-   * Append to the end of the file. 
-   * @param src path of the file being created.
-   * @param clientName name of the current client.
-   * @param flag indicates whether the data is appended to a new block.
-   * @return wrapper with information about the last partial block and file
-   *    status if any
-   * @throws AccessControlException if permission to append file is 
-   * denied by the system. As usually on the client side the exception will 
-   * be wrapped into {@link org.apache.hadoop.ipc.RemoteException}.
-   * Allows appending to an existing file if the server is
-   * configured with the parameter dfs.support.append set to true, otherwise
-   * throws an IOException.
-   * 
-   * @throws AccessControlException If permission to append to file is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws DSQuotaExceededException If append violates disk space quota 
-   *           restriction
-   * @throws SafeModeException append not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred.
-   *
-   * RuntimeExceptions:
-   * @throws UnsupportedOperationException if append is not supported
-   */
-  @AtMostOnce
-  public LastBlockWithStatus append(String src, String clientName,
-      EnumSetWritable<CreateFlag> flag) throws AccessControlException,
-      DSQuotaExceededException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Set replication for an existing file.
-   * <p>
-   * The NameNode sets replication to the new value and returns.
-   * The actual block replication is not expected to be performed during  
-   * this method call. The blocks will be populated or removed in the 
-   * background as the result of the routine block maintenance procedures.
-   * 
-   * @param src file name
-   * @param replication new replication
-   * 
-   * @return true if successful;
-   *         false if file does not exist or is a directory
-   *
-   * @throws AccessControlException If access is denied
-   * @throws DSQuotaExceededException If replication violates disk space 
-   *           quota restriction
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException not allowed in safemode
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public boolean setReplication(String src, short replication)
-      throws AccessControlException, DSQuotaExceededException,
-      FileNotFoundException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
-
-  /**
-   * Get all the available block storage policies.
-   * @return All the in-use block storage policies currently.
-   */
-  @Idempotent
-  public BlockStoragePolicy[] getStoragePolicies() throws IOException;
-
-  /**
-   * Set the storage policy for a file/directory
-   * @param src Path of an existing file/directory. 
-   * @param policyName The name of the storage policy
-   * @throws SnapshotAccessControlException If access is denied
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
-   * @throws FileNotFoundException If file/dir <code>src</code> is not found
-   * @throws QuotaExceededException If changes violate the quota restriction
-   */
-  @Idempotent
-  public void setStoragePolicy(String src, String policyName)
-      throws SnapshotAccessControlException, UnresolvedLinkException,
-      FileNotFoundException, QuotaExceededException, IOException;
-
-  /**
-   * Set permissions for an existing file/directory.
-   * 
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void setPermission(String src, FsPermission permission)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Set Owner of a path (i.e. a file or a directory).
-   * The parameters username and groupname cannot both be null.
-   * @param src file path
-   * @param username If it is null, the original username remains unchanged.
-   * @param groupname If it is null, the original groupname remains unchanged.
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void setOwner(String src, String username, String groupname)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * The client can give up on a block by calling abandonBlock().
-   * The client can then either obtain a new block, or complete or abandon the 
-   * file.
-   * Any partial writes to the block will be discarded.
-   * 
-   * @param b         Block to abandon
-   * @param fileId    The id of the file where the block resides.  Older clients
-   *                    will pass GRANDFATHER_INODE_ID here.
-   * @param src       The path of the file where the block resides.
-   * @param holder    Lease holder.
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void abandonBlock(ExtendedBlock b, long fileId,
-      String src, String holder)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * A client that wants to write an additional block to the 
-   * indicated filename (which must currently be open for writing)
-   * should call addBlock().  
-   *
-   * addBlock() allocates a new block and datanodes the block data
-   * should be replicated to.
-   * 
-   * addBlock() also commits the previous block by reporting
-   * to the name-node the actual generation stamp and the length
-   * of the block that the client has transmitted to data-nodes.
-   *
-   * @param src the file being created
-   * @param clientName the name of the client that adds the block
-   * @param previous  previous block
-   * @param excludeNodes a list of nodes that should not be
-   * allocated for the current block
-   * @param fileId the id uniquely identifying a file
-   * @param favoredNodes the list of nodes where the client wants the blocks.
-   *          Nodes are identified by either host name or address.
-   *
-   * @return LocatedBlock allocated block information.
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws NotReplicatedYetException previous blocks of the file are not
-   *           replicated yet. Blocks cannot be added until replication
-   *           completes.
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public LocatedBlock addBlock(String src, String clientName,
-      ExtendedBlock previous, DatanodeInfo[] excludeNodes, long fileId, 
-      String[] favoredNodes)
-      throws AccessControlException, FileNotFoundException,
-      NotReplicatedYetException, SafeModeException, UnresolvedLinkException,
-      IOException;
-
-  /** 
-   * Get a datanode for an existing pipeline.
-   * 
-   * @param src the file being written
-   * @param fileId the ID of the file being written
-   * @param blk the block being written
-   * @param existings the existing nodes in the pipeline
-   * @param excludes the excluded nodes
-   * @param numAdditionalNodes number of additional datanodes
-   * @param clientName the name of the client
-   * 
-   * @return the located block.
-   * 
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public LocatedBlock getAdditionalDatanode(final String src,
-      final long fileId, final ExtendedBlock blk,
-      final DatanodeInfo[] existings,
-      final String[] existingStorageIDs,
-      final DatanodeInfo[] excludes,
-      final int numAdditionalNodes, final String clientName
-      ) throws AccessControlException, FileNotFoundException,
-          SafeModeException, UnresolvedLinkException, IOException;
-
-  /**
-   * The client is done writing data to the given filename, and would 
-   * like to complete it.  
-   *
-   * The function returns whether the file has been closed successfully.
-   * If the function returns false, the caller should try again.
-   * 
-   * close() also commits the last block of file by reporting
-   * to the name-node the actual generation stamp and the length
-   * of the block that the client has transmitted to data-nodes.
-   *
-   * A call to complete() will not return true until all the file's
-   * blocks have been replicated the minimum number of times.  Thus,
-   * DataNode failures may cause a client to call complete() several
-   * times before succeeding.
-   *
-   * @param src the file being created
-   * @param clientName the name of the client that adds the block
-   * @param last the last block info
-   * @param fileId the id uniquely identifying a file
-   *
-   * @return true if all file blocks are minimally replicated or false otherwise
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink 
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public boolean complete(String src, String clientName,
-                          ExtendedBlock last, long fileId)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * The client wants to report corrupted blocks (blocks with specified
-   * locations on datanodes).
-   * @param blocks Array of located blocks to report
-   */
-  @Idempotent
-  public void reportBadBlocks(LocatedBlock[] blocks) throws IOException;
-
-  ///////////////////////////////////////
-  // Namespace management
-  ///////////////////////////////////////
-  /**
-   * Rename an item in the file system namespace.
-   * @param src existing file or directory name.
-   * @param dst new name.
-   * @return true if successful, or false if the old name does not exist
-   * or if the new name already belongs to the namespace.
-   * 
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException an I/O error occurred 
-   */
-  @AtMostOnce
-  public boolean rename(String src, String dst) 
-      throws UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Moves blocks from srcs to trg and delete srcs
-   * 
-   * @param trg existing file
-   * @param srcs - list of existing files (same block size, same replication)
-   * @throws IOException if some arguments are invalid
-   * @throws UnresolvedLinkException if <code>trg</code> or <code>srcs</code>
-   *           contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   */
-  @AtMostOnce
-  public void concat(String trg, String[] srcs) 
-      throws IOException, UnresolvedLinkException, SnapshotAccessControlException;
-
-  /**
-   * Rename src to dst.
-   * <ul>
-   * <li>Fails if src is a file and dst is a directory.
-   * <li>Fails if src is a directory and dst is a file.
-   * <li>Fails if the parent of dst does not exist or is a file.
-   * </ul>
-   * <p>
-   * Without OVERWRITE option, rename fails if the dst already exists.
-   * With OVERWRITE option, rename overwrites the dst, if it is a file 
-   * or an empty directory. Rename fails if dst is a non-empty directory.
-   * <p>
-   * This implementation of rename is atomic.
-   * <p>
-   * @param src existing file or directory name.
-   * @param dst new name.
-   * @param options Rename options
-   * 
-   * @throws AccessControlException If access is denied
-   * @throws DSQuotaExceededException If rename violates disk space 
-   *           quota restriction
-   * @throws FileAlreadyExistsException If <code>dst</code> already exists and
-   *           <code>options</options> has {@link Rename#OVERWRITE} option
-   *           false.
-   * @throws FileNotFoundException If <code>src</code> does not exist
-   * @throws NSQuotaExceededException If rename violates namespace 
-   *           quota restriction
-   * @throws ParentNotDirectoryException If parent of <code>dst</code> 
-   *           is not a directory
-   * @throws SafeModeException rename not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> or
-   *           <code>dst</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @AtMostOnce
-  public void rename2(String src, String dst, Options.Rename... options)
-      throws AccessControlException, DSQuotaExceededException,
-      FileAlreadyExistsException, FileNotFoundException,
-      NSQuotaExceededException, ParentNotDirectoryException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Truncate file src to new size.
-   * <ul>
-   * <li>Fails if src is a directory.
-   * <li>Fails if src does not exist.
-   * <li>Fails if src is not closed.
-   * <li>Fails if new size is greater than current size.
-   * </ul>
-   * <p>
-   * This implementation of truncate is purely a namespace operation if truncate
-   * occurs at a block boundary. Requires DataNode block recovery otherwise.
-   * <p>
-   * @param src  existing file
-   * @param newLength  the target size
-   *
-   * @return true if client does not need to wait for block recovery,
-   * false if client needs to wait for block recovery.
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException truncate not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public boolean truncate(String src, long newLength, String clientName)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Delete the given file or directory from the file system.
-   * <p>
-   * same as delete but provides a way to avoid accidentally 
-   * deleting non empty directories programmatically. 
-   * @param src existing name
-   * @param recursive if true deletes a non empty directory recursively,
-   * else throws an exception.
-   * @return true only if the existing file or directory was actually removed 
-   * from the file system.
-   * 
-   * @throws AccessControlException If access is denied
-   * @throws FileNotFoundException If file <code>src</code> is not found
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @AtMostOnce
-  public boolean delete(String src, boolean recursive)
-      throws AccessControlException, FileNotFoundException, SafeModeException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-  
-  /**
-   * Create a directory (or hierarchy of directories) with the given
-   * name and permission.
-   *
-   * @param src The path of the directory being created
-   * @param masked The masked permission of the directory being created
-   * @param createParent create missing parent directory if true
-   *
-   * @return True if the operation success.
-   *
-   * @throws AccessControlException If access is denied
-   * @throws FileAlreadyExistsException If <code>src</code> already exists
-   * @throws FileNotFoundException If parent of <code>src</code> does not exist
-   *           and <code>createParent</code> is false
-   * @throws NSQuotaExceededException If file creation violates quota restriction
-   * @throws ParentNotDirectoryException If parent of <code>src</code> 
-   *           is not a directory
-   * @throws SafeModeException create not allowed in safemode
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred.
-   *
-   * RunTimeExceptions:
-   * @throws InvalidPathException If <code>src</code> is invalid
-   */
-  @Idempotent
-  public boolean mkdirs(String src, FsPermission masked, boolean createParent)
-      throws AccessControlException, FileAlreadyExistsException,
-      FileNotFoundException, NSQuotaExceededException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
-
-  /**
-   * Get a partial listing of the indicated directory
-   *
-   * @param src the directory name
-   * @param startAfter the name to start listing after encoded in java UTF8
-   * @param needLocation if the FileStatus should contain block locations
-   *
-   * @return a partial listing starting after startAfter
-   *
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException If <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public DirectoryListing getListing(String src,
-                                     byte[] startAfter,
-                                     boolean needLocation)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-  
-  /**
-   * Get listing of all the snapshottable directories
-   * 
-   * @return Information about all the current snapshottable directory
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
-      throws IOException;
-
-  ///////////////////////////////////////
-  // System issues and management
-  ///////////////////////////////////////
-
-  /**
-   * Client programs can cause stateful changes in the NameNode
-   * that affect other clients.  A client may obtain a file and 
-   * neither abandon nor complete it.  A client might hold a series
-   * of locks that prevent other clients from proceeding.
-   * Clearly, it would be bad if a client held a bunch of locks
-   * that it never gave up.  This can happen easily if the client
-   * dies unexpectedly.
-   * <p>
-   * So, the NameNode will revoke the locks and live file-creates
-   * for clients that it thinks have died.  A client tells the
-   * NameNode that it is still alive by periodically calling
-   * renewLease().  If a certain amount of time passes since
-   * the last call to renewLease(), the NameNode assumes the
-   * client has died.
-   *
-   * @throws AccessControlException permission denied
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void renewLease(String clientName) throws AccessControlException,
-      IOException;
-
-  /**
-   * Start lease recovery.
-   * Lightweight NameNode operation to trigger lease recovery
-   * 
-   * @param src path of the file to start lease recovery
-   * @param clientName name of the current client
-   * @return true if the file is already closed
-   * @throws IOException
-   */
-  @Idempotent
-  public boolean recoverLease(String src, String clientName) throws IOException;
-
-  public int GET_STATS_CAPACITY_IDX = 0;
-  public int GET_STATS_USED_IDX = 1;
-  public int GET_STATS_REMAINING_IDX = 2;
-  public int GET_STATS_UNDER_REPLICATED_IDX = 3;
-  public int GET_STATS_CORRUPT_BLOCKS_IDX = 4;
-  public int GET_STATS_MISSING_BLOCKS_IDX = 5;
-  public int GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX = 6;
-  
-  /**
-   * Get a set of statistics about the filesystem.
-   * Right now, only seven values are returned.
-   * <ul>
-   * <li> [0] contains the total storage capacity of the system, in bytes.</li>
-   * <li> [1] contains the total used space of the system, in bytes.</li>
-   * <li> [2] contains the available storage of the system, in bytes.</li>
-   * <li> [3] contains number of under replicated blocks in the system.</li>
-   * <li> [4] contains number of blocks with a corrupt replica. </li>
-   * <li> [5] contains number of blocks without any good replicas left. </li>
-   * <li> [6] contains number of blocks which have replication factor
-   *          1 and have lost the only replica. </li>
-   * </ul>
-   * Use public constants like {@link #GET_STATS_CAPACITY_IDX} in place of 
-   * actual numbers to index into the array.
-   */
-  @Idempotent
-  public long[] getStats() throws IOException;
-
-  /**
-   * Get a report on the system's current datanodes.
-   * One DatanodeInfo object is returned for each DataNode.
-   * Return live datanodes if type is LIVE; dead datanodes if type is DEAD;
-   * otherwise all datanodes if type is ALL.
-   */
-  @Idempotent
-  public DatanodeInfo[] getDatanodeReport(HdfsConstants.DatanodeReportType type)
-      throws IOException;
-
-  /**
-   * Get a report on the current datanode storages.
-   */
-  @Idempotent
-  public DatanodeStorageReport[] getDatanodeStorageReport(
-      HdfsConstants.DatanodeReportType type) throws IOException;
-
-  /**
-   * Get the block size for the given file.
-   * @param filename The name of the file
-   * @return The number of bytes in each block
-   * @throws IOException
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   */
-  @Idempotent
-  public long getPreferredBlockSize(String filename) 
-      throws IOException, UnresolvedLinkException;
-
-  /**
-   * Enter, leave or get safe mode.
-   * <p>
-   * Safe mode is a name node state when it
-   * <ol><li>does not accept changes to name space (read-only), and</li>
-   * <li>does not replicate or delete blocks.</li></ol>
-   * 
-   * <p>
-   * Safe mode is entered automatically at name node startup.
-   * Safe mode can also be entered manually using
-   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}.
-   * <p>
-   * At startup the name node accepts data node reports collecting
-   * information about block locations.
-   * In order to leave safe mode it needs to collect a configurable
-   * percentage called threshold of blocks, which satisfy the minimal 
-   * replication condition.
-   * The minimal replication condition is that each block must have at least
-   * <tt>dfs.namenode.replication.min</tt> replicas.
-   * When the threshold is reached the name node extends safe mode
-   * for a configurable amount of time
-   * to let the remaining data nodes to check in before it
-   * will start replicating missing blocks.
-   * Then the name node leaves safe mode.
-   * <p>
-   * If safe mode is turned on manually using
-   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_ENTER,false)}
-   * then the name node stays in safe mode until it is manually turned off
-   * using {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_LEAVE,false)}.
-   * Current state of the name node can be verified using
-   * {@link #setSafeMode(HdfsConstants.SafeModeAction,boolean) setSafeMode(SafeModeAction.SAFEMODE_GET,false)}
-   * <h4>Configuration parameters:</h4>
-   * <tt>dfs.safemode.threshold.pct</tt> is the threshold parameter.<br>
-   * <tt>dfs.safemode.extension</tt> is the safe mode extension parameter.<br>
-   * <tt>dfs.namenode.replication.min</tt> is the minimal replication parameter.
-   * 
-   * <h4>Special cases:</h4>
-   * The name node does not enter safe mode at startup if the threshold is 
-   * set to 0 or if the name space is empty.<br>
-   * If the threshold is set to 1 then all blocks need to have at least 
-   * minimal replication.<br>
-   * If the threshold value is greater than 1 then the name node will not be 
-   * able to turn off safe mode automatically.<br>
-   * Safe mode can always be turned off manually.
-   * 
-   * @param action  <ul> <li>0 leave safe mode;</li>
-   *                <li>1 enter safe mode;</li>
-   *                <li>2 get safe mode state.</li></ul>
-   * @param isChecked If true then action will be done only in ActiveNN.
-   * 
-   * @return <ul><li>0 if the safe mode is OFF or</li> 
-   *         <li>1 if the safe mode is ON.</li></ul>
-   *                   
-   * @throws IOException
-   */
-  @Idempotent
-  public boolean setSafeMode(HdfsConstants.SafeModeAction action, boolean isChecked) 
-      throws IOException;
-
-  /**
-   * Save namespace image.
-   * <p>
-   * Saves current namespace into storage directories and reset edits log.
-   * Requires superuser privilege and safe mode.
-   *
-   * @param timeWindow NameNode does a checkpoint if the latest checkpoint was
-   *                   done beyond the given time period (in seconds).
-   * @param txGap NameNode does a checkpoint if the gap between the latest
-   *              checkpoint and the latest transaction id is greater this gap.
-   * @return whether an extra checkpoint has been done
-   *
-   * @throws IOException if image creation failed.
-   */
-  @AtMostOnce
-  public boolean saveNamespace(long timeWindow, long txGap) throws IOException;
-
-  
-  /**
-   * Roll the edit log.
-   * Requires superuser privileges.
-   * 
-   * @throws AccessControlException if the superuser privilege is violated
-   * @throws IOException if log roll fails
-   * @return the txid of the new segment
-   */
-  @Idempotent
-  public long rollEdits() throws AccessControlException, IOException;
-
-  /**
-   * Enable/Disable restore failed storage.
-   * <p>
-   * sets flag to enable restore of failed storage replicas
-   * 
-   * @throws AccessControlException if the superuser privilege is violated.
-   */
-  @Idempotent
-  public boolean restoreFailedStorage(String arg) 
-      throws AccessControlException, IOException;
-
-  /**
-   * Tells the namenode to reread the hosts and exclude files. 
-   * @throws IOException
-   */
-  @Idempotent
-  public void refreshNodes() throws IOException;
-
-  /**
-   * Finalize previous upgrade.
-   * Remove file system state saved during the upgrade.
-   * The upgrade will become irreversible.
-   * 
-   * @throws IOException
-   */
-  @Idempotent
-  public void finalizeUpgrade() throws IOException;
-
-  /**
-   * Rolling upgrade operations.
-   * @param action either query, prepare or finalize.
-   * @return rolling upgrade information.
-   */
-  @Idempotent
-  public RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action)
-      throws IOException;
-
-  /**
-   * @return CorruptFileBlocks, containing a list of corrupt files (with
-   *         duplicates if there is more than one corrupt block in a file)
-   *         and a cookie
-   * @throws IOException
-   *
-   * Each call returns a subset of the corrupt files in the system. To obtain
-   * all corrupt files, call this method repeatedly and each time pass in the
-   * cookie returned from the previous call.
-   */
-  @Idempotent
-  public CorruptFileBlocks listCorruptFileBlocks(String path, String cookie)
-      throws IOException;
-  
-  /**
-   * Dumps namenode data structures into specified file. If the file
-   * already exists, then append.
-   *
-   * @throws IOException
-   */
-  @Idempotent
-  public void metaSave(String filename) throws IOException;
-
-  /**
-   * Tell all datanodes to use a new, non-persistent bandwidth value for
-   * dfs.balance.bandwidthPerSec.
-   *
-   * @param bandwidth Blanacer bandwidth in bytes per second for this datanode.
-   * @throws IOException
-   */
-  @Idempotent
-  public void setBalancerBandwidth(long bandwidth) throws IOException;
-  
-  /**
-   * Get the file info for a specific file or directory.
-   * @param src The string representation of the path to the file
-   *
-   * @return object containing information regarding the file
-   *         or null if file not found
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException if the path contains a symlink. 
-   * @throws IOException If an I/O error occurred        
-   */
-  @Idempotent
-  public HdfsFileStatus getFileInfo(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException;
-  
-  /**
-   * Get the close status of a file
-   * @param src The string representation of the path to the file
-   *
-   * @return return true if file is closed
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException if the path contains a symlink.
-   * @throws IOException If an I/O error occurred     
-   */
-  @Idempotent
-  public boolean isFileClosed(String src) throws AccessControlException,
-      FileNotFoundException, UnresolvedLinkException, IOException;
-  
-  /**
-   * Get the file info for a specific file or directory. If the path 
-   * refers to a symlink then the FileStatus of the symlink is returned.
-   * @param src The string representation of the path to the file
-   *
-   * @return object containing information regarding the file
-   *         or null if file not found
-   *
-   * @throws AccessControlException permission denied
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink
-   * @throws IOException If an I/O error occurred        
-   */
-  @Idempotent
-  public HdfsFileStatus getFileLinkInfo(String src)
-      throws AccessControlException, UnresolvedLinkException, IOException;
-  
-  /**
-   * Get {@link ContentSummary} rooted at the specified directory.
-   * @param path The string representation of the path
-   *
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>path</code> is not found
-   * @throws UnresolvedLinkException if <code>path</code> contains a symlink. 
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public ContentSummary getContentSummary(String path)
-      throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, IOException;
-
-  /**
-   * Set the quota for a directory.
-   * @param path  The string representation of the path to the directory
-   * @param namespaceQuota Limit on the number of names in the tree rooted 
-   *                       at the directory
-   * @param storagespaceQuota Limit on storage space occupied all the files under
-   *                       this directory.
-   * @param type StorageType that the space quota is intended to be set on.
-   *             It may be null when called by traditional space/namespace quota.
-   *             When type is is not null, the storagespaceQuota parameter is for
-   *             type specified and namespaceQuota must be
-   *             {@link HdfsConstants#QUOTA_DONT_SET}.
-   *
-   * <br><br>
-   *                       
-   * The quota can have three types of values : (1) 0 or more will set 
-   * the quota to that value, (2) {@link HdfsConstants#QUOTA_DONT_SET}  implies
-   * the quota will not be changed, and (3) {@link HdfsConstants#QUOTA_RESET}
-   * implies the quota will be reset. Any other value is a runtime error.
-   * 
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>path</code> is not found
-   * @throws QuotaExceededException if the directory size 
-   *           is greater than the given quota
-   * @throws UnresolvedLinkException if the <code>path</code> contains a symlink. 
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
-      StorageType type) throws AccessControlException, FileNotFoundException,
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Write all metadata for this file into persistent storage.
-   * The file must be currently open for writing.
-   * @param src The string representation of the path
-   * @param inodeId The inode ID, or GRANDFATHER_INODE_ID if the client is
-   *                too old to support fsync with inode IDs.
-   * @param client The string representation of the client
-   * @param lastBlockLength The length of the last block (under construction) 
-   *                        to be reported to NameNode 
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void fsync(String src, long inodeId, String client,
-                    long lastBlockLength)
-      throws AccessControlException, FileNotFoundException, 
-      UnresolvedLinkException, IOException;
-
-  /**
-   * Sets the modification and access time of the file to the specified time.
-   * @param src The string representation of the path
-   * @param mtime The number of milliseconds since Jan 1, 1970.
-   *              Setting mtime to -1 means that modification time should not be set
-   *              by this call.
-   * @param atime The number of milliseconds since Jan 1, 1970.
-   *              Setting atime to -1 means that access time should not be set
-   *              by this call.
-   *              
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException file <code>src</code> is not found
-   * @throws UnresolvedLinkException if <code>src</code> contains a symlink. 
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @Idempotent
-  public void setTimes(String src, long mtime, long atime)
-      throws AccessControlException, FileNotFoundException, 
-      UnresolvedLinkException, SnapshotAccessControlException, IOException;
-
-  /**
-   * Create symlink to a file or directory.
-   * @param target The path of the destination that the
-   *               link points to.
-   * @param link The path of the link being created.
-   * @param dirPerm permissions to use when creating parent directories
-   * @param createParent - if true then missing parent dirs are created
-   *                       if false then parent must exist
-   *
-   * @throws AccessControlException permission denied
-   * @throws FileAlreadyExistsException If file <code>link</code> already exists
-   * @throws FileNotFoundException If parent of <code>link</code> does not exist
-   *           and <code>createParent</code> is false
-   * @throws ParentNotDirectoryException If parent of <code>link</code> is not a
-   *           directory.
-   * @throws UnresolvedLinkException if <code>link</target> contains a symlink. 
-   * @throws SnapshotAccessControlException if path is in RO snapshot
-   * @throws IOException If an I/O error occurred
-   */
-  @AtMostOnce
-  public void createSymlink(String target, String link, FsPermission dirPerm,
-      boolean createParent) throws AccessControlException,
-      FileAlreadyExistsException, FileNotFoundException,
-      ParentNotDirectoryException, SafeModeException, UnresolvedLinkException,
-      SnapshotAccessControlException, IOException;
-
-  /**
-   * Return the target of the given symlink. If there is an intermediate
-   * symlink in the path (ie a symlink leading up to the final path component)
-   * then the given path is returned with this symlink resolved.
-   *
-   * @param path The path with a link that needs resolution.
-   * @return The path after resolving the first symbolic link in the path.
-   * @throws AccessControlException permission denied
-   * @throws FileNotFoundException If <code>path</code> does not exist
-   * @throws IOException If the given path does not refer to a symlink
-   *           or an I/O error occurred
-   */
-  @Idempotent
-  public String getLinkTarget(String path) throws AccessControlException,
-      FileNotFoundException, IOException; 
-  
-  /**
-   * Get a new generation stamp together with an access token for 
-   * a block under construction
-   * 
-   * This method is called only when a client needs to recover a failed
-   * pipeline or set up a pipeline for appending to a block.
-   * 
-   * @param block a block
-   * @param clientName the name of the client
-   * @return a located block with a new generation stamp and an access token
-   * @throws IOException if any error occurs
-   */
-  @Idempotent
-  public LocatedBlock updateBlockForPipeline(ExtendedBlock block,
-      String clientName) throws IOException;
-
-  /**
-   * Update a pipeline for a block under construction
-   * 
-   * @param clientName the name of the client
-   * @param oldBlock the old block
-   * @param newBlock the new block containing new generation stamp and length
-   * @param newNodes datanodes in the pipeline
-   * @throws IOException if any error occurs
-   */
-  @AtMostOnce
-  public void updatePipeline(String clientName, ExtendedBlock oldBlock, 
-      ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs)
-      throws IOException;
-
-  /**
-   * Get a valid Delegation Token.
-   * 
-   * @param renewer the designated renewer for the token
-   * @return Token<DelegationTokenIdentifier>
-   * @throws IOException
-   */
-  @Idempotent
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer) 
-      throws IOException;
-
-  /**
-   * Renew an existing delegation token.
-   * 
-   * @param token delegation token obtained earlier
-   * @return the new expiration time
-   * @throws IOException
-   */
-  @Idempotent
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException;
-  
-  /**
-   * Cancel an existing delegation token.
-   * 
-   * @param token delegation token
-   * @throws IOException
-   */
-  @Idempotent
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException;
-  
-  /**
-   * @return encryption key so a client can encrypt data sent via the
-   *         DataTransferProtocol to/from DataNodes.
-   * @throws IOException
-   */
-  @Idempotent
-  public DataEncryptionKey getDataEncryptionKey() throws IOException;
-  
-  /**
-   * Create a snapshot
-   * @param snapshotRoot the path that is being snapshotted
-   * @param snapshotName name of the snapshot created
-   * @return the snapshot path.
-   * @throws IOException
-   */
-  @AtMostOnce
-  public String createSnapshot(String snapshotRoot, String snapshotName)
-      throws IOException;
-
-  /**
-   * Delete a specific snapshot of a snapshottable directory
-   * @param snapshotRoot  The snapshottable directory
-   * @param snapshotName Name of the snapshot for the snapshottable directory
-   * @throws IOException
-   */
-  @AtMostOnce
-  public void deleteSnapshot(String snapshotRoot, String snapshotName)
-      throws IOException;
-  
-  /**
-   * Rename a snapshot
-   * @param snapshotRoot the directory path where the snapshot was taken 
-   * @param snapshotOldName old name of the snapshot
-   * @param snapshotNewName new name of the snapshot
-   * @throws IOException
-   */
-  @AtMostOnce
-  public void renameSnapshot(String snapshotRoot, String snapshotOldName,
-      String snapshotNewName) throws IOException;
-  
-  /**
-   * Allow snapshot on a directory.
-   * @param snapshotRoot the directory to be snapped
-   * @throws IOException on error
-   */
-  @Idempotent
-  public void allowSnapshot(String snapshotRoot)
-      throws IOException;
-    
-  /**
-   * Disallow snapshot on a directory.
-   * @param snapshotRoot the directory to disallow snapshot
-   * @throws IOException on error
-   */
-  @Idempotent
-  public void disallowSnapshot(String snapshotRoot)
-      throws IOException;
-  
-  /**
-   * Get the difference between two snapshots, or between a snapshot and the
-   * current tree of a directory.
-   * 
-   * @param snapshotRoot
-   *          full path of the directory where snapshots are taken
-   * @param fromSnapshot
-   *          snapshot name of the from point. Null indicates the current
-   *          tree
-   * @param toSnapshot
-   *          snapshot name of the to point. Null indicates the current
-   *          tree.
-   * @return The difference report represented as a {@link SnapshotDiffReport}.
-   * @throws IOException on error
-   */
-  @Idempotent
-  public SnapshotDiffReport getSnapshotDiffReport(String snapshotRoot,
-      String fromSnapshot, String toSnapshot) throws IOException;
-
-  /**
-   * Add a CacheDirective to the CacheManager.
-   * 
-   * @param directive A CacheDirectiveInfo to be added
-   * @param flags {@link CacheFlag}s to use for this operation.
-   * @return A CacheDirectiveInfo associated with the added directive
-   * @throws IOException if the directive could not be added
-   */
-  @AtMostOnce
-  public long addCacheDirective(CacheDirectiveInfo directive,
-      EnumSet<CacheFlag> flags) throws IOException;
-
-  /**
-   * Modify a CacheDirective in the CacheManager.
-   * 
-   * @param flags {@link CacheFlag}s to use for this operation.
-   * @throws IOException if the directive could not be modified
-   */
-  @AtMostOnce
-  public void modifyCacheDirective(CacheDirectiveInfo directive,
-      EnumSet<CacheFlag> flags) throws IOException;
-
-  /**
-   * Remove a CacheDirectiveInfo from the CacheManager.
-   * 
-   * @param id of a CacheDirectiveInfo
-   * @throws IOException if the cache directive could not be removed
-   */
-  @AtMostOnce
-  public void removeCacheDirective(long id) throws IOException;
-
-  /**
-   * List the set of cached paths of a cache pool. Incrementally fetches results
-   * from the server.
-   * 
-   * @param prevId The last listed entry ID, or -1 if this is the first call to
-   *               listCacheDirectives.
-   * @param filter Parameters to use to filter the list results, 
-   *               or null to display all directives visible to us.
-   * @return A batch of CacheDirectiveEntry objects.
-   */
-  @Idempotent
-  public BatchedEntries<CacheDirectiveEntry> listCacheDirectives(
-      long prevId, CacheDirectiveInfo filter) throws IOException;
-
-  /**
-   * Add a new cache pool.
-   * 
-   * @param info Description of the new cache pool
-   * @throws IOException If the request could not be completed.
-   */
-  @AtMostOnce
-  public void addCachePool(CachePoolInfo info) throws IOException;
-
-  /**
-   * Modify an existing cache pool.
-   *
-   * @param req
-   *          The request to modify a cache pool.
-   * @throws IOException 
-   *          If the request could not be completed.
-   */
-  @AtMostOnce
-  public void modifyCachePool(CachePoolInfo req) throws IOException;
-  
-  /**
-   * Remove a cache pool.
-   * 
-   * @param pool name of the cache pool to remove.
-   * @throws IOException if the cache pool did not exist, or could not be
-   *           removed.
-   */
-  @AtMostOnce
-  public void removeCachePool(String pool) throws IOException;
-
-  /**
-   * List the set of cache pools. Incrementally fetches results from the server.
-   * 
-   * @param prevPool name of the last pool listed, or the empty string if this is
-   *          the first invocation of listCachePools
-   * @return A batch of CachePoolEntry objects.
-   */
-  @Idempotent
-  public BatchedEntries<CachePoolEntry> listCachePools(String prevPool)
-      throws IOException;
-
-  /**
-   * Modifies ACL entries of files and directories.  This method can add new ACL
-   * entries or modify the permissions on existing ACL entries.  All existing
-   * ACL entries that are not specified in this call are retained without
-   * changes.  (Modifications are merged into the current ACL.)
-   */
-  @Idempotent
-  public void modifyAclEntries(String src, List<AclEntry> aclSpec)
-      throws IOException;
-
-  /**
-   * Removes ACL entries from files and directories.  Other ACL entries are
-   * retained.
-   */
-  @Idempotent
-  public void removeAclEntries(String src, List<AclEntry> aclSpec)
-      throws IOException;
-
-  /**
-   * Removes all default ACL entries from files and directories.
-   */
-  @Idempotent
-  public void removeDefaultAcl(String src) throws IOException;
-
-  /**
-   * Removes all but the base ACL entries of files and directories.  The entries
-   * for user, group, and others are retained for compatibility with permission
-   * bits.
-   */
-  @Idempotent
-  public void removeAcl(String src) throws IOException;
-
-  /**
-   * Fully replaces ACL of files and directories, discarding all existing
-   * entries.
-   */
-  @Idempotent
-  public void setAcl(String src, List<AclEntry> aclSpec) throws IOException;
-
-  /**
-   * Gets the ACLs of files and directories.
-   */
-  @Idempotent
-  public AclStatus getAclStatus(String src) throws IOException;
-  
-  /**
-   * Create an encryption zone
-   */
-  @AtMostOnce
-  public void createEncryptionZone(String src, String keyName)
-    throws IOException;
-
-  /**
-   * Get the encryption zone for a path.
-   */
-  @Idempotent
-  public EncryptionZone getEZForPath(String src)
-    throws IOException;
-
-  /**
-   * Used to implement cursor-based batched listing of {@EncryptionZone}s.
-   *
-   * @param prevId ID of the last item in the previous batch. If there is no
-   *               previous batch, a negative value can be used.
-   * @return Batch of encryption zones.
-   */
-  @Idempotent
-  public BatchedEntries<EncryptionZone> listEncryptionZones(
-      long prevId) throws IOException;
-
-  /**
-   * Set xattr of a file or directory.
-   * The name must be prefixed with the namespace followed by ".". For example,
-   * "user.attr".
-   * <p/>
-   * Refer to the HDFS extended attributes user documentation for details.
-   *
-   * @param src file or directory
-   * @param xAttr <code>XAttr</code> to set
-   * @param flag set flag
-   * @throws IOException
-   */
-  @AtMostOnce
-  public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag) 
-      throws IOException;
-  
-  /**
-   * Get xattrs of a file or directory. Values in xAttrs parameter are ignored.
-   * If xAttrs is null or empty, this is the same as getting all xattrs of the
-   * file or directory.  Only those xattrs for which the logged-in user has
-   * permissions to view are returned.
-   * <p/>
-   * Refer to the HDFS extended attributes user documentation for details.
-   *
-   * @param src file or directory
-   * @param xAttrs xAttrs to get
-   * @return List<XAttr> <code>XAttr</code> list 
-   * @throws IOException
-   */
-  @Idempotent
-  public List<XAttr> getXAttrs(String src, List<XAttr> xAttrs) 
-      throws IOException;
-
-  /**
-   * List the xattrs names for a file or directory.
-   * Only the xattr names for which the logged in user has the permissions to
-   * access will be returned.
-   * <p/>
-   * Refer to the HDFS extended attributes user documentation for details.
-   *
-   * @param src file or directory
-   * @return List<XAttr> <code>XAttr</code> list
-   * @throws IOException
-   */
-  @Idempotent
-  public List<XAttr> listXAttrs(String src)
-      throws IOException;
-  
-  /**
-   * Remove xattr of a file or directory.Value in xAttr parameter is ignored.
-   * The name must be prefixed with the namespace followed by ".". For example,
-   * "user.attr".
-   * <p/>
-   * Refer to the HDFS extended attributes user documentation for details.
-   *
-   * @param src file or directory
-   * @param xAttr <code>XAttr</code> to remove
-   * @throws IOException
-   */
-  @AtMostOnce
-  public void removeXAttr(String src, XAttr xAttr) throws IOException;
-
-  /**
-   * Checks if the user can access a path.  The mode specifies which access
-   * checks to perform.  If the requested permissions are granted, then the
-   * method returns normally.  If access is denied, then the method throws an
-   * {@link AccessControlException}.
-   * In general, applications should avoid using this method, due to the risk of
-   * time-of-check/time-of-use race conditions.  The permissions on a file may
-   * change immediately after the access call returns.
-   *
-   * @param path Path to check
-   * @param mode type of access to check
-   * @throws AccessControlException if access is denied
-   * @throws FileNotFoundException if the path does not exist
-   * @throws IOException see specific implementation
-   */
-  @Idempotent
-  public void checkAccess(String path, FsAction mode) throws IOException;
-
-  /**
-   * Get the highest txid the NameNode knows has been written to the edit
-   * log, or -1 if the NameNode's edit log is not yet open for write. Used as
-   * the starting point for the inotify event stream.
-   */
-  @Idempotent
-  public long getCurrentEditLogTxid() throws IOException;
-
-  /**
-   * Get an ordered list of batches of events corresponding to the edit log
-   * transactions for txids equal to or greater than txid.
-   */
-  @Idempotent
-  public EventBatchList getEditsFromTxid(long txid) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eb2ccade/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
deleted file mode 100644
index 8080bcf..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeModeException.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.server.namenode;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * This exception is thrown when the name node is in safe mode.
- * Client cannot modified namespace until the safe mode is off. 
- * 
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class SafeModeException extends IOException {
-  private static final long serialVersionUID = 1L;
-  public SafeModeException(String msg) {
-    super(msg);
-  }
-}
\ No newline at end of file


[42/50] [abbrv] hadoop git commit: YARN-3804. Both RM are on standBy state when kerberos user not in yarn.admin.acl. Contributed by Varun Saxena

Posted by zj...@apache.org.
YARN-3804. Both RM are on standBy state when kerberos user not in yarn.admin.acl. Contributed by Varun Saxena


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c033209e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c033209e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c033209e

Branch: refs/heads/YARN-2928
Commit: c033209e1af3999dd44dfa2f598f2b45cd271a72
Parents: c10bcc0
Author: Xuan <xg...@apache.org>
Authored: Wed Jun 17 16:23:27 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:19:01 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../server/resourcemanager/AdminService.java    | 19 +++++---
 .../resourcemanager/TestRMAdminService.java     | 49 +++++++++++++++++++-
 3 files changed, 63 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c033209e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index ae9716c..6bf0620 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -781,6 +781,9 @@ Release 2.7.1 - UNRELEASED
     YARN-3764. CapacityScheduler should forbid moving LeafQueue from one parent
     to another. (Wangda Tan via jianhe)
 
+    YARN-3804. Both RM are on standBy state when kerberos user not in yarn.admin.acl
+    (Varun Saxena via xgong)
+
 Release 2.7.0 - 2015-04-20
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c033209e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index 1ee8b3b..e5bb6e5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -112,6 +112,8 @@ public class AdminService extends CompositeService implements
   private final RecordFactory recordFactory = 
     RecordFactoryProvider.getRecordFactory(null);
 
+  private UserGroupInformation daemonUser;
+
   @VisibleForTesting
   boolean isDistributedNodeLabelConfiguration = false;
 
@@ -138,10 +140,9 @@ public class AdminService extends CompositeService implements
         YarnConfiguration.RM_ADMIN_ADDRESS,
         YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
         YarnConfiguration.DEFAULT_RM_ADMIN_PORT);
+    daemonUser = UserGroupInformation.getCurrentUser();
     authorizer = YarnAuthorizationProvider.getInstance(conf);
-    authorizer.setAdmins(new AccessControlList(conf.get(
-      YarnConfiguration.YARN_ADMIN_ACL,
-        YarnConfiguration.DEFAULT_YARN_ADMIN_ACL)), UserGroupInformation
+    authorizer.setAdmins(getAdminAclList(conf), UserGroupInformation
         .getCurrentUser());
     rmId = conf.get(YarnConfiguration.RM_HA_ID);
 
@@ -151,6 +152,14 @@ public class AdminService extends CompositeService implements
     super.serviceInit(conf);
   }
 
+  private AccessControlList getAdminAclList(Configuration conf) {
+    AccessControlList aclList = new AccessControlList(conf.get(
+        YarnConfiguration.YARN_ADMIN_ACL,
+        YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
+    aclList.addUser(daemonUser.getShortUserName());
+    return aclList;
+  }
+
   @Override
   protected void serviceStart() throws Exception {
     startServer();
@@ -470,9 +479,7 @@ public class AdminService extends CompositeService implements
     Configuration conf =
         getConfiguration(new Configuration(false),
             YarnConfiguration.YARN_SITE_CONFIGURATION_FILE);
-    authorizer.setAdmins(new AccessControlList(conf.get(
-      YarnConfiguration.YARN_ADMIN_ACL,
-        YarnConfiguration.DEFAULT_YARN_ADMIN_ACL)), UserGroupInformation
+    authorizer.setAdmins(getAdminAclList(conf), UserGroupInformation
         .getCurrentUser());
     RMAuditLogger.logSuccess(user.getShortUserName(), argName,
         "AdminService");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c033209e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
index fe0b8a8..0a05c91 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java
@@ -38,12 +38,14 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.ha.HAServiceProtocol;
 import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
 import org.apache.hadoop.ha.HAServiceProtocol.StateChangeRequestInfo;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.GroupMappingServiceProvider;
 import org.apache.hadoop.security.Groups;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.authorize.ProxyUsers;
 import org.apache.hadoop.security.authorize.ServiceAuthorizationManager;
+import org.apache.hadoop.yarn.LocalConfigurationProvider;
 import org.apache.hadoop.yarn.api.records.DecommissionType;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.conf.HAUtil;
@@ -208,7 +210,8 @@ public class TestRMAdminService {
         rm.adminService.getAccessControlList().getAclString().trim();
 
     Assert.assertTrue(!aclStringAfter.equals(aclStringBefore));
-    Assert.assertEquals(aclStringAfter, "world:anyone:rwcda");
+    Assert.assertEquals(aclStringAfter, "world:anyone:rwcda," +
+        UserGroupInformation.getCurrentUser().getShortUserName());
   }
 
   @Test
@@ -695,7 +698,8 @@ public class TestRMAdminService {
       String aclStringAfter =
           resourceManager.adminService.getAccessControlList()
               .getAclString().trim();
-      Assert.assertEquals(aclStringAfter, "world:anyone:rwcda");
+      Assert.assertEquals(aclStringAfter, "world:anyone:rwcda," +
+          UserGroupInformation.getCurrentUser().getShortUserName());
 
       // validate values for queue configuration
       CapacityScheduler cs =
@@ -761,6 +765,47 @@ public class TestRMAdminService {
     }
   }
 
+  /* For verifying fix for YARN-3804 */
+  @Test
+  public void testRefreshAclWithDaemonUser() throws Exception {
+    String daemonUser =
+        UserGroupInformation.getCurrentUser().getShortUserName();
+    configuration.set(YarnConfiguration.RM_CONFIGURATION_PROVIDER_CLASS,
+        "org.apache.hadoop.yarn.FileSystemBasedConfigurationProvider");
+
+    uploadDefaultConfiguration();
+    YarnConfiguration yarnConf = new YarnConfiguration();
+    yarnConf.set(YarnConfiguration.YARN_ADMIN_ACL, daemonUser + "xyz");
+    uploadConfiguration(yarnConf, "yarn-site.xml");
+
+    try {
+      rm = new MockRM(configuration);
+      rm.init(configuration);
+      rm.start();
+    } catch(Exception ex) {
+      fail("Should not get any exceptions");
+    }
+
+    assertEquals(daemonUser + "xyz," + daemonUser,
+        rm.adminService.getAccessControlList().getAclString().trim());
+
+    yarnConf = new YarnConfiguration();
+    yarnConf.set(YarnConfiguration.YARN_ADMIN_ACL, daemonUser + "abc");
+    uploadConfiguration(yarnConf, "yarn-site.xml");
+    try {
+      rm.adminService.refreshAdminAcls(RefreshAdminAclsRequest.newInstance());
+    } catch (YarnException e) {
+      if (e.getCause() != null &&
+          e.getCause() instanceof AccessControlException) {
+        fail("Refresh should not have failed due to incorrect ACL");
+      }
+      throw e;
+    }
+
+    assertEquals(daemonUser + "abc," + daemonUser,
+        rm.adminService.getAccessControlList().getAclString().trim());
+  }
+
   @Test
   public void testModifyLabelsOnNodesWithDistributedConfigurationDisabled()
       throws IOException, YarnException {


[02/50] [abbrv] hadoop git commit: HDFS-8499. Refactor BlockInfo class hierarchy with static helper class. Contributed by Zhe Zhang.

Posted by zj...@apache.org.
HDFS-8499. Refactor BlockInfo class hierarchy with static helper class. Contributed by Zhe Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/bc9d48ac
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bc9d48ac
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bc9d48ac

Branch: refs/heads/YARN-2928
Commit: bc9d48accbf911662dc40c870a29c51e9ba5b23b
Parents: 98591bd
Author: Andrew Wang <wa...@apache.org>
Authored: Fri Jun 12 11:35:39 2015 -0700
Committer: Zhijie Shen <zj...@apache.org>
Committed: Thu Jun 18 11:10:04 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../server/blockmanagement/BlockCollection.java |   2 +-
 .../hdfs/server/blockmanagement/BlockInfo.java  |  24 +-
 .../blockmanagement/BlockInfoContiguous.java    |  77 +---
 .../BlockInfoContiguousUnderConstruction.java   | 403 ------------------
 .../BlockInfoUnderConstruction.java             | 405 +++++++++++++++++++
 .../BlockInfoUnderConstructionContiguous.java   | 110 +++++
 .../server/blockmanagement/BlockManager.java    |  40 +-
 .../ContiguousBlockStorageOp.java               | 106 +++++
 .../blockmanagement/DatanodeDescriptor.java     |  13 +-
 .../server/blockmanagement/DatanodeManager.java |   4 +-
 .../hdfs/server/namenode/FSDirWriteFileOp.java  |  15 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  15 +-
 .../hdfs/server/namenode/FSImageFormat.java     |   7 +-
 .../server/namenode/FSImageFormatPBINode.java   |   6 +-
 .../server/namenode/FSImageSerialization.java   |   4 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  24 +-
 .../namenode/FileUnderConstructionFeature.java  |  10 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  14 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |   4 +-
 .../server/namenode/snapshot/FileDiffList.java  |   4 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   8 +-
 .../TestBlockInfoUnderConstruction.java         |   6 +-
 .../blockmanagement/TestBlockManager.java       |   6 +-
 .../blockmanagement/TestHeartbeatHandling.java  |   8 +-
 .../blockmanagement/TestReplicationPolicy.java  |   5 +-
 .../namenode/TestBlockUnderConstruction.java    |   4 +-
 .../TestCommitBlockSynchronization.java         |   9 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |   6 +-
 .../namenode/ha/TestRetryCacheWithHA.java       |   6 +-
 .../namenode/snapshot/SnapshotTestHelper.java   |   4 +-
 31 files changed, 769 insertions(+), 583 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index e315db6..033451e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -626,6 +626,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-7923. The DataNodes should rate-limit their full block reports by
     asking the NN on heartbeat messages (cmccabe)
 
+    HDFS-8499. Refactor BlockInfo class hierarchy with static helper class.
+    (Zhe Zhang via wang)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
index 02a1d05..f11a825 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
@@ -79,7 +79,7 @@ public interface BlockCollection {
    * Convert the last block of the collection to an under-construction block
    * and set the locations.
    */
-  public BlockInfoContiguousUnderConstruction setLastBlock(BlockInfo lastBlock,
+  public BlockInfoUnderConstruction setLastBlock(BlockInfo lastBlock,
       DatanodeStorageInfo[] targets) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index dea31c4..4cc2791 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -51,7 +51,7 @@ public abstract class  BlockInfo extends Block
    * per replica is 42 bytes (LinkedList#Entry object per replica) versus 16
    * bytes using the triplets.
    */
-  protected Object[] triplets;
+  Object[] triplets;
 
   /**
    * Construct an entry for blocksmap
@@ -295,7 +295,7 @@ public abstract class  BlockInfo extends Block
   /**
    * BlockInfo represents a block that is not being constructed.
    * In order to start modifying the block, the BlockInfo should be converted
-   * to {@link BlockInfoContiguousUnderConstruction}.
+   * to {@link BlockInfoUnderConstruction}.
    * @return {@link BlockUCState#COMPLETE}
    */
   public BlockUCState getBlockUCState() {
@@ -312,27 +312,29 @@ public abstract class  BlockInfo extends Block
   }
 
   /**
-   * Convert a complete block to an under construction block.
+   * Convert a block to an under construction block.
    * @return BlockInfoUnderConstruction -  an under construction block.
    */
-  public BlockInfoContiguousUnderConstruction convertToBlockUnderConstruction(
+  public BlockInfoUnderConstruction convertToBlockUnderConstruction(
       BlockUCState s, DatanodeStorageInfo[] targets) {
     if(isComplete()) {
-      BlockInfoContiguousUnderConstruction ucBlock =
-          new BlockInfoContiguousUnderConstruction(this,
-          getBlockCollection().getPreferredBlockReplication(), s, targets);
-      ucBlock.setBlockCollection(getBlockCollection());
-      return ucBlock;
+      return convertCompleteBlockToUC(s, targets);
     }
     // the block is already under construction
-    BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction)this;
+    BlockInfoUnderConstruction ucBlock =
+        (BlockInfoUnderConstruction)this;
     ucBlock.setBlockUCState(s);
     ucBlock.setExpectedLocations(targets);
     ucBlock.setBlockCollection(getBlockCollection());
     return ucBlock;
   }
 
+  /**
+   * Convert a complete block to an under construction block.
+   */
+  abstract BlockInfoUnderConstruction convertCompleteBlockToUC(
+      BlockUCState s, DatanodeStorageInfo[] targets);
+
   @Override
   public int hashCode() {
     // Super implementation is sufficient

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index eff89a8..b9abcd0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -19,13 +19,13 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 
 /**
  * Subclass of {@link BlockInfo}, used for a block with replication scheme.
  */
 @InterfaceAudience.Private
 public class BlockInfoContiguous extends BlockInfo {
-  public static final BlockInfoContiguous[] EMPTY_ARRAY = {};
 
   public BlockInfoContiguous(short size) {
     super(size);
@@ -40,84 +40,37 @@ public class BlockInfoContiguous extends BlockInfo {
    * This is used to convert BlockReplicationInfoUnderConstruction
    * @param from BlockReplicationInfo to copy from.
    */
-  protected BlockInfoContiguous(BlockInfoContiguous from) {
+  protected BlockInfoContiguous(BlockInfo from) {
     super(from);
   }
 
-  /**
-   * Ensure that there is enough  space to include num more triplets.
-   * @return first free triplet index.
-   */
-  private int ensureCapacity(int num) {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    int last = numNodes();
-    if (triplets.length >= (last+num)*3) {
-      return last;
-    }
-    /* Not enough space left. Create a new array. Should normally
-     * happen only when replication is manually increased by the user. */
-    Object[] old = triplets;
-    triplets = new Object[(last+num)*3];
-    System.arraycopy(old, 0, triplets, 0, last * 3);
-    return last;
-  }
-
   @Override
   boolean addStorage(DatanodeStorageInfo storage) {
-    // find the last null node
-    int lastNode = ensureCapacity(1);
-    setStorageInfo(lastNode, storage);
-    setNext(lastNode, null);
-    setPrevious(lastNode, null);
-    return true;
+    return ContiguousBlockStorageOp.addStorage(this, storage);
   }
 
   @Override
   boolean removeStorage(DatanodeStorageInfo storage) {
-    int dnIndex = findStorageInfo(storage);
-    if (dnIndex < 0) { // the node is not found
-      return false;
-    }
-    assert getPrevious(dnIndex) == null && getNext(dnIndex) == null :
-        "Block is still in the list and must be removed first.";
-    // find the last not null node
-    int lastNode = numNodes()-1;
-    // replace current node triplet by the lastNode one
-    setStorageInfo(dnIndex, getStorageInfo(lastNode));
-    setNext(dnIndex, getNext(lastNode));
-    setPrevious(dnIndex, getPrevious(lastNode));
-    // set the last triplet to null
-    setStorageInfo(lastNode, null);
-    setNext(lastNode, null);
-    setPrevious(lastNode, null);
-    return true;
+    return ContiguousBlockStorageOp.removeStorage(this, storage);
   }
 
   @Override
   public int numNodes() {
-    assert this.triplets != null : "BlockInfo is not initialized";
-    assert triplets.length % 3 == 0 : "Malformed BlockInfo";
-
-    for (int idx = getCapacity()-1; idx >= 0; idx--) {
-      if (getDatanode(idx) != null) {
-        return idx + 1;
-      }
-    }
-    return 0;
+    return ContiguousBlockStorageOp.numNodes(this);
   }
 
   @Override
   void replaceBlock(BlockInfo newBlock) {
-    assert newBlock instanceof BlockInfoContiguous;
-    for (int i = this.numNodes() - 1; i >= 0; i--) {
-      final DatanodeStorageInfo storage = this.getStorageInfo(i);
-      final boolean removed = storage.removeBlock(this);
-      assert removed : "currentBlock not found.";
+    ContiguousBlockStorageOp.replaceBlock(this, newBlock);
+  }
 
-      final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
-          newBlock);
-      assert result == DatanodeStorageInfo.AddBlockResult.ADDED :
-          "newBlock already exists.";
-    }
+  @Override
+  BlockInfoUnderConstruction convertCompleteBlockToUC(
+      HdfsServerConstants.BlockUCState s, DatanodeStorageInfo[] targets) {
+    BlockInfoUnderConstructionContiguous ucBlock =
+        new BlockInfoUnderConstructionContiguous(this,
+            getBlockCollection().getPreferredBlockReplication(), s, targets);
+    ucBlock.setBlockCollection(getBlockCollection());
+    return ucBlock;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
deleted file mode 100644
index b757171..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
+++ /dev/null
@@ -1,403 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.blockmanagement;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-
-/**
- * Represents a block that is currently being constructed.<br>
- * This is usually the last block of a file opened for write or append.
- */
-public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
-  /** Block state. See {@link BlockUCState} */
-  private BlockUCState blockUCState;
-
-  /**
-   * Block replicas as assigned when the block was allocated.
-   * This defines the pipeline order.
-   */
-  private List<ReplicaUnderConstruction> replicas;
-
-  /**
-   * Index of the primary data node doing the recovery. Useful for log
-   * messages.
-   */
-  private int primaryNodeIndex = -1;
-
-  /**
-   * The new generation stamp, which this block will have
-   * after the recovery succeeds. Also used as a recovery id to identify
-   * the right recovery if any of the abandoned recoveries re-appear.
-   */
-  private long blockRecoveryId = 0;
-
-  /**
-   * The block source to use in the event of copy-on-write truncate.
-   */
-  private Block truncateBlock;
-
-  /**
-   * ReplicaUnderConstruction contains information about replicas while
-   * they are under construction.
-   * The GS, the length and the state of the replica is as reported by 
-   * the data-node.
-   * It is not guaranteed, but expected, that data-nodes actually have
-   * corresponding replicas.
-   */
-  static class ReplicaUnderConstruction extends Block {
-    private final DatanodeStorageInfo expectedLocation;
-    private ReplicaState state;
-    private boolean chosenAsPrimary;
-
-    ReplicaUnderConstruction(Block block,
-                             DatanodeStorageInfo target,
-                             ReplicaState state) {
-      super(block);
-      this.expectedLocation = target;
-      this.state = state;
-      this.chosenAsPrimary = false;
-    }
-
-    /**
-     * Expected block replica location as assigned when the block was allocated.
-     * This defines the pipeline order.
-     * It is not guaranteed, but expected, that the data-node actually has
-     * the replica.
-     */
-    private DatanodeStorageInfo getExpectedStorageLocation() {
-      return expectedLocation;
-    }
-
-    /**
-     * Get replica state as reported by the data-node.
-     */
-    ReplicaState getState() {
-      return state;
-    }
-
-    /**
-     * Whether the replica was chosen for recovery.
-     */
-    boolean getChosenAsPrimary() {
-      return chosenAsPrimary;
-    }
-
-    /**
-     * Set replica state.
-     */
-    void setState(ReplicaState s) {
-      state = s;
-    }
-
-    /**
-     * Set whether this replica was chosen for recovery.
-     */
-    void setChosenAsPrimary(boolean chosenAsPrimary) {
-      this.chosenAsPrimary = chosenAsPrimary;
-    }
-
-    /**
-     * Is data-node the replica belongs to alive.
-     */
-    boolean isAlive() {
-      return expectedLocation.getDatanodeDescriptor().isAlive;
-    }
-
-    @Override // Block
-    public int hashCode() {
-      return super.hashCode();
-    }
-
-    @Override // Block
-    public boolean equals(Object obj) {
-      // Sufficient to rely on super's implementation
-      return (this == obj) || super.equals(obj);
-    }
-
-    @Override
-    public String toString() {
-      final StringBuilder b = new StringBuilder(50);
-      appendStringTo(b);
-      return b.toString();
-    }
-    
-    @Override
-    public void appendStringTo(StringBuilder sb) {
-      sb.append("ReplicaUC[")
-        .append(expectedLocation)
-        .append("|")
-        .append(state)
-        .append("]");
-    }
-  }
-
-  /**
-   * Create block and set its state to
-   * {@link BlockUCState#UNDER_CONSTRUCTION}.
-   */
-  public BlockInfoContiguousUnderConstruction(Block blk, short replication) {
-    this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
-  }
-
-  /**
-   * Create a block that is currently being constructed.
-   */
-  public BlockInfoContiguousUnderConstruction(Block blk, short replication,
-      BlockUCState state, DatanodeStorageInfo[] targets) {
-    super(blk, replication);
-    assert getBlockUCState() != BlockUCState.COMPLETE :
-      "BlockInfoUnderConstruction cannot be in COMPLETE state";
-    this.blockUCState = state;
-    setExpectedLocations(targets);
-  }
-
-  /**
-   * Convert an under construction block to a complete block.
-   * 
-   * @return BlockInfo - a complete block.
-   * @throws IOException if the state of the block 
-   * (the generation stamp and the length) has not been committed by 
-   * the client or it does not have at least a minimal number of replicas 
-   * reported from data-nodes. 
-   */
-  BlockInfo convertToCompleteBlock() throws IOException {
-    assert getBlockUCState() != BlockUCState.COMPLETE :
-      "Trying to convert a COMPLETE block";
-    return new BlockInfoContiguous(this);
-  }
-
-  /** Set expected locations */
-  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
-    int numLocations = targets == null ? 0 : targets.length;
-    this.replicas = new ArrayList<ReplicaUnderConstruction>(numLocations);
-    for(int i = 0; i < numLocations; i++)
-      replicas.add(
-        new ReplicaUnderConstruction(this, targets[i], ReplicaState.RBW));
-  }
-
-  /**
-   * Create array of expected replica locations
-   * (as has been assigned by chooseTargets()).
-   */
-  public DatanodeStorageInfo[] getExpectedStorageLocations() {
-    int numLocations = replicas == null ? 0 : replicas.size();
-    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
-    for(int i = 0; i < numLocations; i++)
-      storages[i] = replicas.get(i).getExpectedStorageLocation();
-    return storages;
-  }
-
-  /** Get the number of expected locations */
-  public int getNumExpectedLocations() {
-    return replicas == null ? 0 : replicas.size();
-  }
-
-  /**
-   * Return the state of the block under construction.
-   * @see BlockUCState
-   */
-  @Override // BlockInfo
-  public BlockUCState getBlockUCState() {
-    return blockUCState;
-  }
-
-  void setBlockUCState(BlockUCState s) {
-    blockUCState = s;
-  }
-
-  /** Get block recovery ID */
-  public long getBlockRecoveryId() {
-    return blockRecoveryId;
-  }
-
-  /** Get recover block */
-  public Block getTruncateBlock() {
-    return truncateBlock;
-  }
-
-  public void setTruncateBlock(Block recoveryBlock) {
-    this.truncateBlock = recoveryBlock;
-  }
-
-  /**
-   * Process the recorded replicas. When about to commit or finish the
-   * pipeline recovery sort out bad replicas.
-   * @param genStamp  The final generation stamp for the block.
-   */
-  public void setGenerationStampAndVerifyReplicas(long genStamp) {
-    // Set the generation stamp for the block.
-    setGenerationStamp(genStamp);
-    if (replicas == null)
-      return;
-
-    // Remove the replicas with wrong gen stamp.
-    // The replica list is unchanged.
-    for (ReplicaUnderConstruction r : replicas) {
-      if (genStamp != r.getGenerationStamp()) {
-        r.getExpectedStorageLocation().removeBlock(this);
-        NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
-            + "from location: {}", r.getExpectedStorageLocation());
-      }
-    }
-  }
-
-  /**
-   * Commit block's length and generation stamp as reported by the client.
-   * Set block state to {@link BlockUCState#COMMITTED}.
-   * @param block - contains client reported block length and generation 
-   * @throws IOException if block ids are inconsistent.
-   */
-  void commitBlock(Block block) throws IOException {
-    if(getBlockId() != block.getBlockId())
-      throw new IOException("Trying to commit inconsistent block: id = "
-          + block.getBlockId() + ", expected id = " + getBlockId());
-    blockUCState = BlockUCState.COMMITTED;
-    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
-    // Sort out invalid replicas.
-    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
-  }
-
-  /**
-   * Initialize lease recovery for this block.
-   * Find the first alive data-node starting from the previous primary and
-   * make it primary.
-   */
-  public void initializeBlockRecovery(long recoveryId) {
-    setBlockUCState(BlockUCState.UNDER_RECOVERY);
-    blockRecoveryId = recoveryId;
-    if (replicas.size() == 0) {
-      NameNode.blockStateChangeLog.warn("BLOCK*"
-        + " BlockInfoUnderConstruction.initLeaseRecovery:"
-        + " No blocks found, lease removed.");
-    }
-    boolean allLiveReplicasTriedAsPrimary = true;
-    for (int i = 0; i < replicas.size(); i++) {
-      // Check if all replicas have been tried or not.
-      if (replicas.get(i).isAlive()) {
-        allLiveReplicasTriedAsPrimary =
-            (allLiveReplicasTriedAsPrimary && replicas.get(i).getChosenAsPrimary());
-      }
-    }
-    if (allLiveReplicasTriedAsPrimary) {
-      // Just set all the replicas to be chosen whether they are alive or not.
-      for (int i = 0; i < replicas.size(); i++) {
-        replicas.get(i).setChosenAsPrimary(false);
-      }
-    }
-    long mostRecentLastUpdate = 0;
-    ReplicaUnderConstruction primary = null;
-    primaryNodeIndex = -1;
-    for(int i = 0; i < replicas.size(); i++) {
-      // Skip alive replicas which have been chosen for recovery.
-      if (!(replicas.get(i).isAlive() && !replicas.get(i).getChosenAsPrimary())) {
-        continue;
-      }
-      final ReplicaUnderConstruction ruc = replicas.get(i);
-      final long lastUpdate = ruc.getExpectedStorageLocation()
-          .getDatanodeDescriptor().getLastUpdateMonotonic();
-      if (lastUpdate > mostRecentLastUpdate) {
-        primaryNodeIndex = i;
-        primary = ruc;
-        mostRecentLastUpdate = lastUpdate;
-      }
-    }
-    if (primary != null) {
-      primary.getExpectedStorageLocation().getDatanodeDescriptor().addBlockToBeRecovered(this);
-      primary.setChosenAsPrimary(true);
-      NameNode.blockStateChangeLog.info(
-          "BLOCK* {} recovery started, primary={}", this, primary);
-    }
-  }
-
-  void addReplicaIfNotPresent(DatanodeStorageInfo storage,
-                     Block block,
-                     ReplicaState rState) {
-    Iterator<ReplicaUnderConstruction> it = replicas.iterator();
-    while (it.hasNext()) {
-      ReplicaUnderConstruction r = it.next();
-      DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
-      if(expectedLocation == storage) {
-        // Record the gen stamp from the report
-        r.setGenerationStamp(block.getGenerationStamp());
-        return;
-      } else if (expectedLocation != null &&
-                 expectedLocation.getDatanodeDescriptor() ==
-                     storage.getDatanodeDescriptor()) {
-
-        // The Datanode reported that the block is on a different storage
-        // than the one chosen by BlockPlacementPolicy. This can occur as
-        // we allow Datanodes to choose the target storage. Update our
-        // state by removing the stale entry and adding a new one.
-        it.remove();
-        break;
-      }
-    }
-    replicas.add(new ReplicaUnderConstruction(block, storage, rState));
-  }
-
-  @Override // BlockInfo
-  // BlockInfoUnderConstruction participates in maps the same way as BlockInfo
-  public int hashCode() {
-    return super.hashCode();
-  }
-
-  @Override // BlockInfo
-  public boolean equals(Object obj) {
-    // Sufficient to rely on super's implementation
-    return (this == obj) || super.equals(obj);
-  }
-
-  @Override
-  public String toString() {
-    final StringBuilder b = new StringBuilder(100);
-    appendStringTo(b);
-    return b.toString();
-  }
-
-  @Override
-  public void appendStringTo(StringBuilder sb) {
-    super.appendStringTo(sb);
-    appendUCParts(sb);
-  }
-
-  private void appendUCParts(StringBuilder sb) {
-    sb.append("{UCState=").append(blockUCState)
-      .append(", truncateBlock=" + truncateBlock)
-      .append(", primaryNodeIndex=").append(primaryNodeIndex)
-      .append(", replicas=[");
-    if (replicas != null) {
-      Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
-      if (iter.hasNext()) {
-        iter.next().appendStringTo(sb);
-        while (iter.hasNext()) {
-          sb.append(", ");
-          iter.next().appendStringTo(sb);
-        }
-      }
-    }
-    sb.append("]}");
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
new file mode 100644
index 0000000..9cd3987
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
@@ -0,0 +1,405 @@
+/**
+ * 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.blockmanagement;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+
+/**
+ * Represents a block that is currently being constructed.<br>
+ * This is usually the last block of a file opened for write or append.
+ */
+public abstract class BlockInfoUnderConstruction extends BlockInfo {
+  /** Block state. See {@link BlockUCState} */
+  protected BlockUCState blockUCState;
+
+  /**
+   * Block replicas as assigned when the block was allocated.
+   * This defines the pipeline order.
+   */
+  protected List<ReplicaUnderConstruction> replicas;
+
+  /**
+   * Index of the primary data node doing the recovery. Useful for log
+   * messages.
+   */
+  private int primaryNodeIndex = -1;
+
+  /**
+   * The new generation stamp, which this block will have
+   * after the recovery succeeds. Also used as a recovery id to identify
+   * the right recovery if any of the abandoned recoveries re-appear.
+   */
+  private long blockRecoveryId = 0;
+
+  /**
+   * The block source to use in the event of copy-on-write truncate.
+   */
+  protected Block truncateBlock;
+
+  /**
+   * ReplicaUnderConstruction contains information about replicas while
+   * they are under construction.
+   * The GS, the length and the state of the replica is as reported by
+   * the data-node.
+   * It is not guaranteed, but expected, that data-nodes actually have
+   * corresponding replicas.
+   */
+  static class ReplicaUnderConstruction extends Block {
+    private final DatanodeStorageInfo expectedLocation;
+    private ReplicaState state;
+    private boolean chosenAsPrimary;
+
+    ReplicaUnderConstruction(Block block,
+                             DatanodeStorageInfo target,
+                             ReplicaState state) {
+      super(block);
+      this.expectedLocation = target;
+      this.state = state;
+      this.chosenAsPrimary = false;
+    }
+
+    /**
+     * Expected block replica location as assigned when the block was allocated.
+     * This defines the pipeline order.
+     * It is not guaranteed, but expected, that the data-node actually has
+     * the replica.
+     */
+    private DatanodeStorageInfo getExpectedStorageLocation() {
+      return expectedLocation;
+    }
+
+    /**
+     * Get replica state as reported by the data-node.
+     */
+    ReplicaState getState() {
+      return state;
+    }
+
+    /**
+     * Whether the replica was chosen for recovery.
+     */
+    boolean getChosenAsPrimary() {
+      return chosenAsPrimary;
+    }
+
+    /**
+     * Set replica state.
+     */
+    void setState(ReplicaState s) {
+      state = s;
+    }
+
+    /**
+     * Set whether this replica was chosen for recovery.
+     */
+    void setChosenAsPrimary(boolean chosenAsPrimary) {
+      this.chosenAsPrimary = chosenAsPrimary;
+    }
+
+    /**
+     * Is data-node the replica belongs to alive.
+     */
+    boolean isAlive() {
+      return expectedLocation.getDatanodeDescriptor().isAlive;
+    }
+
+    @Override // Block
+    public int hashCode() {
+      return super.hashCode();
+    }
+
+    @Override // Block
+    public boolean equals(Object obj) {
+      // Sufficient to rely on super's implementation
+      return (this == obj) || super.equals(obj);
+    }
+
+    @Override
+    public String toString() {
+      final StringBuilder b = new StringBuilder(50);
+      appendStringTo(b);
+      return b.toString();
+    }
+
+    @Override
+    public void appendStringTo(StringBuilder sb) {
+      sb.append("ReplicaUC[")
+        .append(expectedLocation)
+        .append("|")
+        .append(state)
+        .append("]");
+    }
+  }
+
+  /**
+   * Create block and set its state to
+   * {@link BlockUCState#UNDER_CONSTRUCTION}.
+   */
+  public BlockInfoUnderConstruction(Block blk, short replication) {
+    this(blk, replication, BlockUCState.UNDER_CONSTRUCTION, null);
+  }
+
+  /**
+   * Create a block that is currently being constructed.
+   */
+  public BlockInfoUnderConstruction(Block blk, short replication,
+      BlockUCState state, DatanodeStorageInfo[] targets) {
+    super(blk, replication);
+    Preconditions.checkState(getBlockUCState() != BlockUCState.COMPLETE,
+        "BlockInfoUnderConstruction cannot be in COMPLETE state");
+    this.blockUCState = state;
+    setExpectedLocations(targets);
+  }
+
+  /** Set expected locations. */
+  public abstract void setExpectedLocations(DatanodeStorageInfo[] targets);
+
+  /**
+   * Create array of expected replica locations
+   * (as has been assigned by chooseTargets()).
+   */
+  public DatanodeStorageInfo[] getExpectedStorageLocations() {
+    int numLocations = replicas == null ? 0 : replicas.size();
+    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
+    for(int i = 0; i < numLocations; i++) {
+      storages[i] = replicas.get(i).getExpectedStorageLocation();
+    }
+    return storages;
+  }
+
+  /** Get the number of expected locations. */
+  public int getNumExpectedLocations() {
+    return replicas == null ? 0 : replicas.size();
+  }
+
+  /**
+   * Return the state of the block under construction.
+   * @see BlockUCState
+   */
+  @Override // BlockInfo
+  public BlockUCState getBlockUCState() {
+    return blockUCState;
+  }
+
+  void setBlockUCState(BlockUCState s) {
+    blockUCState = s;
+  }
+
+  /** Get block recovery ID. */
+  public long getBlockRecoveryId() {
+    return blockRecoveryId;
+  }
+
+  /** Get recover block. */
+  public abstract Block getTruncateBlock();
+
+  public abstract void setTruncateBlock(Block recoveryBlock);
+
+  /**
+   * Process the recorded replicas. When about to commit or finish the
+   * pipeline recovery sort out bad replicas.
+   * @param genStamp  The final generation stamp for the block.
+   */
+  public void setGenerationStampAndVerifyReplicas(long genStamp) {
+    // Set the generation stamp for the block.
+    setGenerationStamp(genStamp);
+    if (replicas == null) {
+      return;
+    }
+
+    // Remove the replicas with wrong gen stamp.
+    // The replica list is unchanged.
+    for (ReplicaUnderConstruction r : replicas) {
+      if (genStamp != r.getGenerationStamp()) {
+        r.getExpectedStorageLocation().removeBlock(this);
+        NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
+            + "from location: {}", r.getExpectedStorageLocation());
+      }
+    }
+  }
+
+  /**
+   * Commit block's length and generation stamp as reported by the client.
+   * Set block state to {@link BlockUCState#COMMITTED}.
+   * @param block - contains client reported block length and generation
+   * @throws IOException if block ids are inconsistent.
+   */
+  void commitBlock(Block block) throws IOException {
+    if(getBlockId() != block.getBlockId()) {
+      throw new IOException("Trying to commit inconsistent block: id = "
+          + block.getBlockId() + ", expected id = " + getBlockId());
+    }
+    blockUCState = BlockUCState.COMMITTED;
+    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
+    // Sort out invalid replicas.
+    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
+  }
+
+  /**
+   * Initialize lease recovery for this block.
+   * Find the first alive data-node starting from the previous primary and
+   * make it primary.
+   */
+  public void initializeBlockRecovery(long recoveryId) {
+    setBlockUCState(BlockUCState.UNDER_RECOVERY);
+    blockRecoveryId = recoveryId;
+    if (replicas.size() == 0) {
+      NameNode.blockStateChangeLog.warn("BLOCK* " +
+          "BlockInfoUnderConstruction.initLeaseRecovery: " +
+          "No blocks found, lease removed.");
+    }
+    boolean allLiveReplicasTriedAsPrimary = true;
+    for (int i = 0; i < replicas.size(); i++) {
+      // Check if all replicas have been tried or not.
+      if (replicas.get(i).isAlive()) {
+        allLiveReplicasTriedAsPrimary =
+            (allLiveReplicasTriedAsPrimary &&
+                replicas.get(i).getChosenAsPrimary());
+      }
+    }
+    if (allLiveReplicasTriedAsPrimary) {
+      // Just set all the replicas to be chosen whether they are alive or not.
+      for (int i = 0; i < replicas.size(); i++) {
+        replicas.get(i).setChosenAsPrimary(false);
+      }
+    }
+    long mostRecentLastUpdate = 0;
+    ReplicaUnderConstruction primary = null;
+    primaryNodeIndex = -1;
+    for(int i = 0; i < replicas.size(); i++) {
+      // Skip alive replicas which have been chosen for recovery.
+      if (!(replicas.get(i).isAlive() &&
+          !replicas.get(i).getChosenAsPrimary())) {
+        continue;
+      }
+      final ReplicaUnderConstruction ruc = replicas.get(i);
+      final long lastUpdate = ruc.getExpectedStorageLocation()
+          .getDatanodeDescriptor().getLastUpdateMonotonic();
+      if (lastUpdate > mostRecentLastUpdate) {
+        primaryNodeIndex = i;
+        primary = ruc;
+        mostRecentLastUpdate = lastUpdate;
+      }
+    }
+    if (primary != null) {
+      primary.getExpectedStorageLocation().
+          getDatanodeDescriptor().addBlockToBeRecovered(this);
+      primary.setChosenAsPrimary(true);
+      NameNode.blockStateChangeLog.info(
+          "BLOCK* {} recovery started, primary={}", this, primary);
+    }
+  }
+
+  void addReplicaIfNotPresent(DatanodeStorageInfo storage,
+                     Block block,
+                     ReplicaState rState) {
+    Iterator<ReplicaUnderConstruction> it = replicas.iterator();
+    while (it.hasNext()) {
+      ReplicaUnderConstruction r = it.next();
+      DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
+      if(expectedLocation == storage) {
+        // Record the gen stamp from the report
+        r.setGenerationStamp(block.getGenerationStamp());
+        return;
+      } else if (expectedLocation != null &&
+                 expectedLocation.getDatanodeDescriptor() ==
+                     storage.getDatanodeDescriptor()) {
+
+        // The Datanode reported that the block is on a different storage
+        // than the one chosen by BlockPlacementPolicy. This can occur as
+        // we allow Datanodes to choose the target storage. Update our
+        // state by removing the stale entry and adding a new one.
+        it.remove();
+        break;
+      }
+    }
+    replicas.add(new ReplicaUnderConstruction(block, storage, rState));
+  }
+
+  /**
+   * Convert an under construction block to a complete block.
+   *
+   * @return a complete block.
+   * @throws IOException
+   *           if the state of the block (the generation stamp and the length)
+   *           has not been committed by the client or it does not have at
+   *           least a minimal number of replicas reported from data-nodes.
+   */
+  public abstract BlockInfo convertToCompleteBlock();
+
+  @Override
+  BlockInfoUnderConstruction convertCompleteBlockToUC
+      (HdfsServerConstants.BlockUCState s, DatanodeStorageInfo[] targets) {
+    BlockManager.LOG.error("convertCompleteBlockToUC should only be applied " +
+        "on complete blocks.");
+    return null;
+  }
+
+  @Override // BlockInfo
+  // BlockInfoUnderConstruction participates in maps the same way as BlockInfo
+  public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override // BlockInfo
+  public boolean equals(Object obj) {
+    // Sufficient to rely on super's implementation
+    return (this == obj) || super.equals(obj);
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder b = new StringBuilder(100);
+    appendStringTo(b);
+    return b.toString();
+  }
+
+  @Override
+  public void appendStringTo(StringBuilder sb) {
+    super.appendStringTo(sb);
+    appendUCParts(sb);
+  }
+
+  private void appendUCParts(StringBuilder sb) {
+    sb.append("{UCState=").append(blockUCState)
+      .append(", truncateBlock=" + truncateBlock)
+      .append(", primaryNodeIndex=").append(primaryNodeIndex)
+      .append(", replicas=[");
+    if (replicas != null) {
+      Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
+      if (iter.hasNext()) {
+        iter.next().appendStringTo(sb);
+        while (iter.hasNext()) {
+          sb.append(", ");
+          iter.next().appendStringTo(sb);
+        }
+      }
+    }
+    sb.append("]}");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
new file mode 100644
index 0000000..c66675a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstructionContiguous.java
@@ -0,0 +1,110 @@
+/**
+ * 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.blockmanagement;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+
+import java.util.ArrayList;
+
+/**
+ * Subclass of {@link BlockInfoUnderConstruction}, representing a block under
+ * the contiguous (instead of striped) layout.
+ */
+public class BlockInfoUnderConstructionContiguous extends
+    BlockInfoUnderConstruction {
+  /**
+   * Create block and set its state to
+   * {@link HdfsServerConstants.BlockUCState#UNDER_CONSTRUCTION}.
+   */
+  public BlockInfoUnderConstructionContiguous(Block blk, short replication) {
+    this(blk, replication, HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+        null);
+  }
+
+  /**
+   * Create a block that is currently being constructed.
+   */
+  public BlockInfoUnderConstructionContiguous(Block blk, short replication,
+      HdfsServerConstants.BlockUCState state, DatanodeStorageInfo[] targets) {
+    super(blk, replication);
+    Preconditions.checkState(getBlockUCState() !=
+        HdfsServerConstants.BlockUCState.COMPLETE,
+        "BlockInfoUnderConstructionContiguous cannot be in COMPLETE state");
+    this.blockUCState = state;
+    setExpectedLocations(targets);
+  }
+
+  /**
+   * Convert an under construction block to a complete block.
+   *
+   * @return BlockInfo - a complete block.
+   * @throws IOException if the state of the block
+   * (the generation stamp and the length) has not been committed by
+   * the client or it does not have at least a minimal number of replicas
+   * reported from data-nodes.
+   */
+  @Override
+  public BlockInfoContiguous convertToCompleteBlock() {
+    Preconditions.checkState(getBlockUCState() !=
+        HdfsServerConstants.BlockUCState.COMPLETE,
+        "Trying to convert a COMPLETE block");
+    return new BlockInfoContiguous(this);
+  }
+
+  @Override
+  boolean addStorage(DatanodeStorageInfo storage) {
+    return ContiguousBlockStorageOp.addStorage(this, storage);
+  }
+
+  @Override
+  boolean removeStorage(DatanodeStorageInfo storage) {
+    return ContiguousBlockStorageOp.removeStorage(this, storage);
+  }
+
+  @Override
+  public int numNodes() {
+    return ContiguousBlockStorageOp.numNodes(this);
+  }
+
+  @Override
+  void replaceBlock(BlockInfo newBlock) {
+    ContiguousBlockStorageOp.replaceBlock(this, newBlock);
+  }
+
+  @Override
+  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
+    int numLocations = targets == null ? 0 : targets.length;
+    this.replicas = new ArrayList<>(numLocations);
+    for(int i = 0; i < numLocations; i++) {
+      replicas.add(
+          new ReplicaUnderConstruction(this, targets[i], HdfsServerConstants.ReplicaState.RBW));
+    }
+  }
+
+  @Override
+  public Block getTruncateBlock() {
+    return truncateBlock;
+  }
+
+  @Override
+  public void setTruncateBlock(Block recoveryBlock) {
+    this.truncateBlock = recoveryBlock;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 4562d94..ebc9017 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -602,7 +602,7 @@ public class BlockManager {
    * of replicas reported from data-nodes.
    */
   private static boolean commitBlock(
-      final BlockInfoContiguousUnderConstruction block, final Block commitBlock)
+      final BlockInfoUnderConstruction block, final Block commitBlock)
       throws IOException {
     if (block.getBlockUCState() == BlockUCState.COMMITTED)
       return false;
@@ -634,7 +634,7 @@ public class BlockManager {
       return false; // already completed (e.g. by syncBlock)
     
     final boolean b = commitBlock(
-        (BlockInfoContiguousUnderConstruction) lastBlock, commitBlock);
+        (BlockInfoUnderConstruction) lastBlock, commitBlock);
     if(countNodes(lastBlock).liveReplicas() >= minReplication)
       completeBlock(bc, bc.numBlocks()-1, false);
     return b;
@@ -654,8 +654,8 @@ public class BlockManager {
     BlockInfo curBlock = bc.getBlocks()[blkIndex];
     if(curBlock.isComplete())
       return curBlock;
-    BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction) curBlock;
+    BlockInfoUnderConstruction ucBlock =
+        (BlockInfoUnderConstruction) curBlock;
     int numNodes = ucBlock.numNodes();
     if (!force && numNodes < minReplication)
       throw new IOException("Cannot complete block: " +
@@ -697,7 +697,7 @@ public class BlockManager {
    * when tailing edit logs as a Standby.
    */
   public BlockInfo forceCompleteBlock(final BlockCollection bc,
-      final BlockInfoContiguousUnderConstruction block) throws IOException {
+      final BlockInfoUnderConstruction block) throws IOException {
     block.commitBlock(block);
     return completeBlock(bc, block, true);
   }
@@ -728,7 +728,7 @@ public class BlockManager {
 
     DatanodeStorageInfo[] targets = getStorages(oldBlock);
 
-    BlockInfoContiguousUnderConstruction ucBlock =
+    BlockInfoUnderConstruction ucBlock =
         bc.setLastBlock(oldBlock, targets);
     blocksMap.replaceBlock(ucBlock);
 
@@ -830,14 +830,14 @@ public class BlockManager {
   /** @return a LocatedBlock for the given block */
   private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos
       ) throws IOException {
-    if (blk instanceof BlockInfoContiguousUnderConstruction) {
+    if (blk instanceof BlockInfoUnderConstruction) {
       if (blk.isComplete()) {
         throw new IOException(
             "blk instanceof BlockInfoUnderConstruction && blk.isComplete()"
             + ", blk=" + blk);
       }
-      final BlockInfoContiguousUnderConstruction uc =
-          (BlockInfoContiguousUnderConstruction) blk;
+      final BlockInfoUnderConstruction uc =
+          (BlockInfoUnderConstruction) blk;
       final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
       final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
       return newLocatedBlock(eb, storages, pos, false);
@@ -1744,11 +1744,11 @@ public class BlockManager {
    * reported by the datanode in the block report. 
    */
   static class StatefulBlockInfo {
-    final BlockInfoContiguousUnderConstruction storedBlock;
+    final BlockInfoUnderConstruction storedBlock;
     final Block reportedBlock;
     final ReplicaState reportedState;
     
-    StatefulBlockInfo(BlockInfoContiguousUnderConstruction storedBlock,
+    StatefulBlockInfo(BlockInfoUnderConstruction storedBlock,
         Block reportedBlock, ReplicaState reportedState) {
       this.storedBlock = storedBlock;
       this.reportedBlock = reportedBlock;
@@ -1789,7 +1789,7 @@ public class BlockManager {
 
     BlockToMarkCorrupt(BlockInfo stored, long gs, String reason,
         Reason reasonCode) {
-      this(new BlockInfoContiguous((BlockInfoContiguous)stored), stored,
+      this(new BlockInfoContiguous(stored), stored,
           reason, reasonCode);
       //the corrupted block in datanode has a different generation stamp
       corrupted.setGenerationStamp(gs);
@@ -2148,13 +2148,13 @@ public class BlockManager {
       
       // If block is under construction, add this replica to its list
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-        ((BlockInfoContiguousUnderConstruction)storedBlock)
+        ((BlockInfoUnderConstruction)storedBlock)
             .addReplicaIfNotPresent(storageInfo, iblk, reportedState);
         // OpenFileBlocks only inside snapshots also will be added to safemode
         // threshold. So we need to update such blocks to safemode
         // refer HDFS-5283
-        BlockInfoContiguousUnderConstruction blockUC =
-            (BlockInfoContiguousUnderConstruction) storedBlock;
+        BlockInfoUnderConstruction blockUC =
+            (BlockInfoUnderConstruction) storedBlock;
         if (namesystem.isInSnapshot(blockUC)) {
           int numOfReplicas = blockUC.getNumExpectedLocations();
           namesystem.incrementSafeBlockCount(numOfReplicas);
@@ -2309,7 +2309,7 @@ public class BlockManager {
 
     if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
       toUC.add(new StatefulBlockInfo(
-          (BlockInfoContiguousUnderConstruction) storedBlock,
+          (BlockInfoUnderConstruction) storedBlock,
           new Block(block), reportedState));
       return storedBlock;
     }
@@ -2500,7 +2500,7 @@ public class BlockManager {
 
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
       DatanodeStorageInfo storageInfo) throws IOException {
-    BlockInfoContiguousUnderConstruction block = ucBlock.storedBlock;
+    BlockInfoUnderConstruction block = ucBlock.storedBlock;
     block.addReplicaIfNotPresent(
         storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
 
@@ -2561,7 +2561,7 @@ public class BlockManager {
     assert block != null && namesystem.hasWriteLock();
     BlockInfo storedBlock;
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
-    if (block instanceof BlockInfoContiguousUnderConstruction) {
+    if (block instanceof BlockInfoUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
       storedBlock = blocksMap.getStoredBlock(block);
     } else {
@@ -3499,8 +3499,8 @@ public class BlockManager {
       String src, BlockInfo[] blocks) {
     for (BlockInfo b: blocks) {
       if (!b.isComplete()) {
-        final BlockInfoContiguousUnderConstruction uc =
-            (BlockInfoContiguousUnderConstruction)b;
+        final BlockInfoUnderConstruction uc =
+            (BlockInfoUnderConstruction)b;
         final int numNodes = b.numNodes();
         LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
           + uc.getBlockUCState() + ", replication# = " + numNodes

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java
new file mode 100644
index 0000000..092f65e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ContiguousBlockStorageOp.java
@@ -0,0 +1,106 @@
+/**
+ * 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.blockmanagement;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Utility class with logic on managing storage locations shared between
+ * complete and under-construction blocks under the contiguous format --
+ * {@link BlockInfoContiguous} and
+ * {@link BlockInfoUnderConstructionContiguous}.
+ */
+class ContiguousBlockStorageOp {
+  /**
+   * Ensure that there is enough  space to include num more triplets.
+   * @return first free triplet index.
+   */
+  private static int ensureCapacity(BlockInfo b, int num) {
+    Preconditions.checkArgument(b.triplets != null,
+        "BlockInfo is not initialized");
+    int last = b.numNodes();
+    if (b.triplets.length >= (last+num)*3) {
+      return last;
+    }
+    /* Not enough space left. Create a new array. Should normally
+     * happen only when replication is manually increased by the user. */
+    Object[] old = b.triplets;
+    b.triplets = new Object[(last+num)*3];
+    System.arraycopy(old, 0, b.triplets, 0, last * 3);
+    return last;
+  }
+
+  static boolean addStorage(BlockInfo b, DatanodeStorageInfo storage) {
+    // find the last null node
+    int lastNode = ensureCapacity(b, 1);
+    b.setStorageInfo(lastNode, storage);
+    b.setNext(lastNode, null);
+    b.setPrevious(lastNode, null);
+    return true;
+  }
+
+  static boolean removeStorage(BlockInfo b,
+      DatanodeStorageInfo storage) {
+    int dnIndex = b.findStorageInfo(storage);
+    if (dnIndex < 0) { // the node is not found
+      return false;
+    }
+    Preconditions.checkArgument(b.getPrevious(dnIndex) == null &&
+            b.getNext(dnIndex) == null,
+        "Block is still in the list and must be removed first.");
+    // find the last not null node
+    int lastNode = b.numNodes()-1;
+    // replace current node triplet by the lastNode one
+    b.setStorageInfo(dnIndex, b.getStorageInfo(lastNode));
+    b.setNext(dnIndex, b.getNext(lastNode));
+    b.setPrevious(dnIndex, b.getPrevious(lastNode));
+    // set the last triplet to null
+    b.setStorageInfo(lastNode, null);
+    b.setNext(lastNode, null);
+    b.setPrevious(lastNode, null);
+    return true;
+  }
+
+  static int numNodes(BlockInfo b) {
+    Preconditions.checkArgument(b.triplets != null,
+        "BlockInfo is not initialized");
+    Preconditions.checkArgument(b.triplets.length % 3 == 0,
+        "Malformed BlockInfo");
+
+    for (int idx = b.getCapacity()-1; idx >= 0; idx--) {
+      if (b.getDatanode(idx) != null) {
+        return idx + 1;
+      }
+    }
+    return 0;
+  }
+
+  static void replaceBlock(BlockInfo b, BlockInfo newBlock) {
+    for (int i = b.numNodes() - 1; i >= 0; i--) {
+      final DatanodeStorageInfo storage = b.getStorageInfo(i);
+      final boolean removed = storage.removeBlock(b);
+      Preconditions.checkState(removed, "currentBlock not found.");
+
+      final DatanodeStorageInfo.AddBlockResult result = storage.addBlock(
+          newBlock);
+      Preconditions.checkState(
+          result == DatanodeStorageInfo.AddBlockResult.ADDED,
+          "newBlock already exists.");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 02abc1d..dd7b301 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -219,8 +219,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /** A queue of blocks to be replicated by this datanode */
   private final BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
   /** A queue of blocks to be recovered by this datanode */
-  private final BlockQueue<BlockInfoContiguousUnderConstruction> recoverBlocks =
-                                new BlockQueue<BlockInfoContiguousUnderConstruction>();
+  private final BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
+                                new BlockQueue<BlockInfoUnderConstruction>();
   /** A set of blocks to be invalidated by this datanode */
   private final LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
 
@@ -599,7 +599,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Store block recovery work.
    */
-  void addBlockToBeRecovered(BlockInfoContiguousUnderConstruction block) {
+  void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
     if(recoverBlocks.contains(block)) {
       // this prevents adding the same block twice to the recovery queue
       BlockManager.LOG.info(block + " is already in the recovery queue");
@@ -641,11 +641,12 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return replicateBlocks.poll(maxTransfers);
   }
 
-  public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
-    List<BlockInfoContiguousUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
+  public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(
+      int maxTransfers) {
+    List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
       return null;
-    return blocks.toArray(new BlockInfoContiguousUnderConstruction[blocks.size()]);
+    return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 58349cc..8143fb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1380,12 +1380,12 @@ public class DatanodeManager {
         }
 
         //check lease recovery
-        BlockInfoContiguousUnderConstruction[] blocks = nodeinfo
+        BlockInfoUnderConstruction[] blocks = nodeinfo
             .getLeaseRecoveryCommand(Integer.MAX_VALUE);
         if (blocks != null) {
           BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
               blocks.length);
-          for (BlockInfoContiguousUnderConstruction b : blocks) {
+          for (BlockInfoUnderConstruction b : blocks) {
             final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             final List<DatanodeStorageInfo> recoveryLocations =

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 3b5f973..4830d5d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -43,7 +43,8 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
@@ -73,7 +74,7 @@ class FSDirWriteFileOp {
       Block block) throws IOException {
     // modify file-> block and blocksMap
     // fileNode should be under construction
-    BlockInfoContiguousUnderConstruction uc = fileNode.removeLastBlock(block);
+    BlockInfoUnderConstruction uc = fileNode.removeLastBlock(block);
     if (uc == null) {
       return false;
     }
@@ -236,7 +237,7 @@ class FSDirWriteFileOp {
       } else {
         // add new chosen targets to already allocated block and return
         BlockInfo lastBlockInFile = pendingFile.getLastBlock();
-        ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
+        ((BlockInfoUnderConstruction) lastBlockInFile)
             .setExpectedLocations(targets);
         offset = pendingFile.computeFileSize();
         return makeLocatedBlock(fsn, lastBlockInFile, targets, offset);
@@ -520,8 +521,8 @@ class FSDirWriteFileOp {
           fileINode.getPreferredBlockReplication(), true);
 
       // associate new last block for the file
-      BlockInfoContiguousUnderConstruction blockInfo =
-        new BlockInfoContiguousUnderConstruction(
+      BlockInfoUnderConstruction blockInfo =
+        new BlockInfoUnderConstructionContiguous(
             block,
             fileINode.getFileReplication(),
             HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
@@ -662,8 +663,8 @@ class FSDirWriteFileOp {
             "allocation of a new block in " + src + ". Returning previously" +
             " allocated block " + lastBlockInFile);
         long offset = file.computeFileSize();
-        BlockInfoContiguousUnderConstruction lastBlockUC =
-            (BlockInfoContiguousUnderConstruction) lastBlockInFile;
+        BlockInfoUnderConstruction lastBlockUC =
+            (BlockInfoUnderConstruction) lastBlockInFile;
         onRetryBlock[0] = makeLocatedBlock(fsn, lastBlockInFile,
             lastBlockUC.getExpectedStorageLocations(), offset);
         return new FileState(file, src, iip);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 3de6760..df01edd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -45,7 +45,8 @@ import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -960,16 +961,16 @@ public class FSEditLogLoader {
       }
       
       oldLastBlock.setNumBytes(pBlock.getNumBytes());
-      if (oldLastBlock instanceof BlockInfoContiguousUnderConstruction) {
+      if (oldLastBlock instanceof BlockInfoUnderConstruction) {
         fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoContiguousUnderConstruction) oldLastBlock);
+            (BlockInfoUnderConstruction) oldLastBlock);
         fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock);
       }
     } else { // the penultimate block is null
       Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
     }
     // add the new block
-    BlockInfo newBI = new BlockInfoContiguousUnderConstruction(
+    BlockInfo newBI = new BlockInfoUnderConstructionContiguous(
           newBlock, file.getPreferredBlockReplication());
     fsNamesys.getBlockManager().addBlockCollection(newBI, file);
     file.addBlock(newBI);
@@ -1010,11 +1011,11 @@ public class FSEditLogLoader {
         oldBlock.getGenerationStamp() != newBlock.getGenerationStamp();
       oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
       
-      if (oldBlock instanceof BlockInfoContiguousUnderConstruction &&
+      if (oldBlock instanceof BlockInfoUnderConstruction &&
           (!isLastBlock || op.shouldCompleteLastBlock())) {
         changeMade = true;
         fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoContiguousUnderConstruction) oldBlock);
+            (BlockInfoUnderConstruction) oldBlock);
       }
       if (changeMade) {
         // The state or gen-stamp of the block has changed. So, we may be
@@ -1049,7 +1050,7 @@ public class FSEditLogLoader {
           // TODO: shouldn't this only be true for the last block?
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
-          newBI = new BlockInfoContiguousUnderConstruction(
+          newBI = new BlockInfoUnderConstructionContiguous(
               newBlock, file.getPreferredBlockReplication());
         } else {
           // OP_CLOSE should add finalized blocks. This code path

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index 30517d0..2305e31 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -54,7 +54,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -777,8 +777,9 @@ public class FSImageFormat {
             // convert the last block to BlockUC
             if (blocks.length > 0) {
               BlockInfo lastBlk = blocks[blocks.length - 1];
-              blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction(
-                  lastBlk, replication);
+              blocks[blocks.length - 1] =
+                  new BlockInfoUnderConstructionContiguous(
+                      lastBlk, replication);
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index e8378e5..d90751c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@@ -364,8 +364,8 @@ public final class FSImageFormatPBINode {
         if (blocks.length > 0) {
           BlockInfo lastBlk = file.getLastBlock();
           // replace the last block of file
-          file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction(
-              lastBlk, replication));
+          file.setBlock(file.numBlocks() - 1,
+              new BlockInfoUnderConstructionContiguous(lastBlk, replication));
         }
       }
       return file;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
index f71cf0b..2dc6252 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -137,7 +137,7 @@ public class FSImageSerialization {
     // last block is UNDER_CONSTRUCTION
     if(numBlocks > 0) {
       blk.readFields(in);
-      blocks[i] = new BlockInfoContiguousUnderConstruction(
+      blocks[i] = new BlockInfoUnderConstructionContiguous(
         blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
     }
     PermissionStatus perm = PermissionStatus.read(in);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index d3b32da..ef53692 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -204,7 +204,8 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstructionContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -2003,7 +2004,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final BlockInfo last = file.getLastBlock();
     if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
       final Block truncateBlock
-          = ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock();
+          = ((BlockInfoUnderConstruction)last).getTruncateBlock();
       if (truncateBlock != null) {
         final long truncateLength = file.computeFileSize(false, false)
             + truncateBlock.getNumBytes();
@@ -2082,11 +2083,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
               nextGenerationStamp(blockIdManager.isLegacyBlock(oldBlock)));
     }
 
-    BlockInfoContiguousUnderConstruction truncatedBlockUC;
+    BlockInfoUnderConstruction truncatedBlockUC;
     if(shouldCopyOnTruncate) {
       // Add new truncateBlock into blocksMap and
       // use oldBlock as a source for copy-on-truncate recovery
-      truncatedBlockUC = new BlockInfoContiguousUnderConstruction(newBlock,
+      truncatedBlockUC = new BlockInfoUnderConstructionContiguous(newBlock,
           file.getPreferredBlockReplication());
       truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
       truncatedBlockUC.setTruncateBlock(oldBlock);
@@ -2102,7 +2103,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
       oldBlock = file.getLastBlock();
       assert !oldBlock.isComplete() : "oldBlock should be under construction";
-      truncatedBlockUC = (BlockInfoContiguousUnderConstruction) oldBlock;
+      truncatedBlockUC = (BlockInfoUnderConstruction) oldBlock;
       truncatedBlockUC.setTruncateBlock(new Block(oldBlock));
       truncatedBlockUC.getTruncateBlock().setNumBytes(
           oldBlock.getNumBytes() - lastBlockDelta);
@@ -3519,7 +3520,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new AlreadyBeingCreatedException(message);
     case UNDER_CONSTRUCTION:
     case UNDER_RECOVERY:
-      final BlockInfoContiguousUnderConstruction uc = (BlockInfoContiguousUnderConstruction)lastBlock;
+      final BlockInfoUnderConstruction uc =
+          (BlockInfoUnderConstruction)lastBlock;
       // determine if last block was intended to be truncated
       Block recoveryBlock = uc.getTruncateBlock();
       boolean truncateRecovery = recoveryBlock != null;
@@ -3635,7 +3637,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
   
   @Override
-  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC) {
+  public boolean isInSnapshot(BlockInfoUnderConstruction blockUC) {
     assert hasReadLock();
     final BlockCollection bc = blockUC.getBlockCollection();
     if (bc == null || !(bc instanceof INodeFile)
@@ -3682,7 +3684,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     waitForLoadingFSImage();
     writeLock();
     boolean copyTruncate = false;
-    BlockInfoContiguousUnderConstruction truncatedBlock = null;
+    BlockInfoUnderConstruction truncatedBlock = null;
     try {
       checkOperation(OperationCategory.WRITE);
       // If a DN tries to commit to the standby, the recovery will
@@ -3739,7 +3741,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         return;
       }
 
-      truncatedBlock = (BlockInfoContiguousUnderConstruction) iFile
+      truncatedBlock = (BlockInfoUnderConstruction) iFile
           .getLastBlock();
       long recoveryId = truncatedBlock.getBlockRecoveryId();
       copyTruncate = truncatedBlock.getBlockId() != storedBlock.getBlockId();
@@ -5774,8 +5776,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     assert hasWriteLock();
     // check the vadility of the block and lease holder name
     final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
-    final BlockInfoContiguousUnderConstruction blockinfo
-        = (BlockInfoContiguousUnderConstruction)pendingFile.getLastBlock();
+    final BlockInfoUnderConstruction blockinfo
+        = (BlockInfoUnderConstruction)pendingFile.getLastBlock();
 
     // check new GS & length: this is not expected
     if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() ||

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
index d07ae1f..74c5d09 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 
 /**
@@ -61,7 +61,7 @@ public class FileUnderConstructionFeature implements INode.Feature {
     BlockInfo lastBlock = f.getLastBlock();
     assert (lastBlock != null) : "The last block for path "
         + f.getFullPathName() + " is null when updating its length";
-    assert (lastBlock instanceof BlockInfoContiguousUnderConstruction)
+    assert (lastBlock instanceof BlockInfoUnderConstruction)
         : "The last block for path " + f.getFullPathName()
             + " is not a BlockInfoUnderConstruction when updating its length";
     lastBlock.setNumBytes(lastBlockLength);
@@ -76,9 +76,9 @@ public class FileUnderConstructionFeature implements INode.Feature {
       final BlocksMapUpdateInfo collectedBlocks) {
     final BlockInfo[] blocks = f.getBlocks();
     if (blocks != null && blocks.length > 0
-        && blocks[blocks.length - 1] instanceof BlockInfoContiguousUnderConstruction) {
-      BlockInfoContiguousUnderConstruction lastUC =
-          (BlockInfoContiguousUnderConstruction) blocks[blocks.length - 1];
+        && blocks[blocks.length - 1] instanceof BlockInfoUnderConstruction) {
+      BlockInfoUnderConstruction lastUC =
+          (BlockInfoUnderConstruction) blocks[blocks.length - 1];
       if (lastUC.getNumBytes() == 0) {
         // this is a 0-sized block. do not need check its UC state here
         collectedBlocks.addDeleteBlock(lastUC);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 294323c..4590eec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -231,7 +231,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override // BlockCollection, the file should be under construction
-  public BlockInfoContiguousUnderConstruction setLastBlock(
+  public BlockInfoUnderConstruction setLastBlock(
       BlockInfo lastBlock, DatanodeStorageInfo[] locations)
       throws IOException {
     Preconditions.checkState(isUnderConstruction(),
@@ -240,7 +240,7 @@ public class INodeFile extends INodeWithAdditionalFields
     if (numBlocks() == 0) {
       throw new IOException("Failed to set last block: File is empty.");
     }
-    BlockInfoContiguousUnderConstruction ucBlock =
+    BlockInfoUnderConstruction ucBlock =
       lastBlock.convertToBlockUnderConstruction(
           BlockUCState.UNDER_CONSTRUCTION, locations);
     setBlock(numBlocks() - 1, ucBlock);
@@ -251,7 +251,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * Remove a block from the block list. This block should be
    * the last one on the list.
    */
-  BlockInfoContiguousUnderConstruction removeLastBlock(Block oldblock) {
+  BlockInfoUnderConstruction removeLastBlock(Block oldblock) {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
     if (blocks == null || blocks.length == 0) {
@@ -262,8 +262,8 @@ public class INodeFile extends INodeWithAdditionalFields
       return null;
     }
 
-    BlockInfoContiguousUnderConstruction uc =
-        (BlockInfoContiguousUnderConstruction)blocks[size_1];
+    BlockInfoUnderConstruction uc =
+        (BlockInfoUnderConstruction)blocks[size_1];
     //copy to a new list
     BlockInfo[] newlist = new BlockInfo[size_1];
     System.arraycopy(blocks, 0, newlist, 0, size_1);
@@ -689,7 +689,7 @@ public class INodeFile extends INodeWithAdditionalFields
     final int last = blocks.length - 1;
     //check if the last block is BlockInfoUnderConstruction
     long size = blocks[last].getNumBytes();
-    if (blocks[last] instanceof BlockInfoContiguousUnderConstruction) {
+    if (blocks[last] instanceof BlockInfoUnderConstruction) {
        if (!includesLastUcBlock) {
          size = 0;
        } else if (usePreferredBlockSize4LastUcBlock) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc9d48ac/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git 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
index 3442e7b..40c4765 100644
--- 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
@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.util.RwLock;
 import org.apache.hadoop.ipc.StandbyException;
@@ -45,5 +45,5 @@ public interface Namesystem extends RwLock, SafeMode {
 
   public void checkOperation(OperationCategory read) throws StandbyException;
 
-  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC);
+  public boolean isInSnapshot(BlockInfoUnderConstruction blockUC);
 }
\ No newline at end of file