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 ka...@apache.org on 2015/03/25 15:54:31 UTC

[01/51] [abbrv] hadoop git commit: HDFS-6841. Use Time.monotonicNow() wherever applicable instead of Time.now(). Contributed by Vinayakumar B

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2139 ddffcd8fa -> c906a1de7


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
index 9b62467..8b2d11e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java
@@ -262,7 +262,7 @@ public class TestBalancer {
   throws IOException, TimeoutException {
     long timeout = TIMEOUT;
     long failtime = (timeout <= 0L) ? Long.MAX_VALUE
-             : Time.now() + timeout;
+             : Time.monotonicNow() + timeout;
     
     while (true) {
       long[] status = client.getStats();
@@ -274,7 +274,7 @@ public class TestBalancer {
           && usedSpaceVariance < CAPACITY_ALLOWED_VARIANCE)
         break; //done
 
-      if (Time.now() > failtime) {
+      if (Time.monotonicNow() > failtime) {
         throw new TimeoutException("Cluster failed to reached expected values of "
             + "totalSpace (current: " + status[0] 
             + ", expected: " + expectedTotalSpace 
@@ -369,7 +369,7 @@ public class TestBalancer {
       int expectedExcludedNodes) throws IOException, TimeoutException {
     long timeout = TIMEOUT;
     long failtime = (timeout <= 0L) ? Long.MAX_VALUE
-        : Time.now() + timeout;
+        : Time.monotonicNow() + timeout;
     if (!p.nodesToBeIncluded.isEmpty()) {
       totalCapacity = p.nodesToBeIncluded.size() * CAPACITY;
     }
@@ -399,7 +399,7 @@ public class TestBalancer {
         }
         if (Math.abs(avgUtilization - nodeUtilization) > BALANCE_ALLOWED_VARIANCE) {
           balanced = false;
-          if (Time.now() > failtime) {
+          if (Time.monotonicNow() > failtime) {
             throw new TimeoutException(
                 "Rebalancing expected avg utilization to become "
                 + avgUtilization + ", but on datanode " + datanode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 f61176e..23e610f 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
@@ -186,7 +186,7 @@ public class BlockManagerTestUtil {
       Assert.assertNotNull("Could not find DN with name: " + dnName, theDND);
       
       synchronized (hbm) {
-        theDND.setLastUpdate(0);
+        DFSTestUtil.setDatanodeDead(theDND);
         hbm.heartbeatCheck();
       }
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 453f411..a7ba293 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,6 +23,7 @@ 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;
 
 /**
@@ -46,40 +47,34 @@ public class TestBlockInfoUnderConstruction {
         new DatanodeStorageInfo[] {s1, s2, s3});
 
     // Recovery attempt #1.
-    long currentTime = System.currentTimeMillis();
-    dd1.setLastUpdate(currentTime - 3 * 1000);
-    dd2.setLastUpdate(currentTime - 1 * 1000);
-    dd3.setLastUpdate(currentTime - 2 * 1000);
+    DFSTestUtil.resetLastUpdatesWithOffset(dd1, -3 * 1000);
+    DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
+    DFSTestUtil.resetLastUpdatesWithOffset(dd3, -2 * 1000);
     blockInfo.initializeBlockRecovery(1);
     BlockInfoContiguousUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
     // Recovery attempt #2.
-    currentTime = System.currentTimeMillis();
-    dd1.setLastUpdate(currentTime - 2 * 1000);
-    dd2.setLastUpdate(currentTime - 1 * 1000);
-    dd3.setLastUpdate(currentTime - 3 * 1000);
+    DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
+    DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
+    DFSTestUtil.resetLastUpdatesWithOffset(dd3, -3 * 1000);
     blockInfo.initializeBlockRecovery(2);
     blockInfoRecovery = dd1.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
     // Recovery attempt #3.
-    currentTime = System.currentTimeMillis();
-    dd1.setLastUpdate(currentTime - 2 * 1000);
-    dd2.setLastUpdate(currentTime - 1 * 1000);
-    dd3.setLastUpdate(currentTime - 3 * 1000);
-    currentTime = System.currentTimeMillis();
+    DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
+    DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
+    DFSTestUtil.resetLastUpdatesWithOffset(dd3, -3 * 1000);
     blockInfo.initializeBlockRecovery(3);
     blockInfoRecovery = dd3.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
     // Recovery attempt #4.
     // Reset everything. And again pick DN with most recent heart beat.
-    currentTime = System.currentTimeMillis();
-    dd1.setLastUpdate(currentTime - 2 * 1000);
-    dd2.setLastUpdate(currentTime - 1 * 1000);
-    dd3.setLastUpdate(currentTime);
-    currentTime = System.currentTimeMillis();
+    DFSTestUtil.resetLastUpdatesWithOffset(dd1, -2 * 1000);
+    DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
+    DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
     blockInfo.initializeBlockRecovery(3);
     blockInfoRecovery = dd3.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 efd1feb..6fc30ba 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
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
@@ -38,6 +39,7 @@ 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;
 
 /**
@@ -164,9 +166,9 @@ public class TestHeartbeatHandling {
           NameNodeAdapter.sendHeartBeat(nodeReg3, dd3, namesystem);
 
           // Test with all alive nodes.
-          dd1.setLastUpdate(System.currentTimeMillis());
-          dd2.setLastUpdate(System.currentTimeMillis());
-          dd3.setLastUpdate(System.currentTimeMillis());
+          DFSTestUtil.resetLastUpdatesWithOffset(dd1, 0);
+          DFSTestUtil.resetLastUpdatesWithOffset(dd2, 0);
+          DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
           final DatanodeStorageInfo[] storages = {
               dd1.getStorageInfos()[0],
               dd2.getStorageInfos()[0],
@@ -189,10 +191,10 @@ public class TestHeartbeatHandling {
           assertEquals(recoveringNodes[2], dd3);
 
           // Test with one stale node.
-          dd1.setLastUpdate(System.currentTimeMillis());
+          DFSTestUtil.resetLastUpdatesWithOffset(dd1, 0);
           // More than the default stale interval of 30 seconds.
-          dd2.setLastUpdate(System.currentTimeMillis() - 40 * 1000);
-          dd3.setLastUpdate(System.currentTimeMillis());
+          DFSTestUtil.resetLastUpdatesWithOffset(dd2, -40 * 1000);
+          DFSTestUtil.resetLastUpdatesWithOffset(dd3, 0);
           blockInfo = new BlockInfoContiguousUnderConstruction(
               new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
               BlockUCState.UNDER_RECOVERY, storages);
@@ -210,10 +212,10 @@ public class TestHeartbeatHandling {
           assertEquals(recoveringNodes[1], dd3);
 
           // Test with all stale node.
-          dd1.setLastUpdate(System.currentTimeMillis() - 60 * 1000);
+          DFSTestUtil.resetLastUpdatesWithOffset(dd1, - 60 * 1000);
           // More than the default stale interval of 30 seconds.
-          dd2.setLastUpdate(System.currentTimeMillis() - 40 * 1000);
-          dd3.setLastUpdate(System.currentTimeMillis() - 80 * 1000);
+          DFSTestUtil.resetLastUpdatesWithOffset(dd2, - 40 * 1000);
+          DFSTestUtil.resetLastUpdatesWithOffset(dd3, - 80 * 1000);
           blockInfo = new BlockInfoContiguousUnderConstruction(
               new Block(0, 0, GenerationStamp.LAST_RESERVED_STAMP), (short) 3,
               BlockUCState.UNDER_RECOVERY, storages);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java
index 5435572..733446c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHostFileManager.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -142,7 +143,7 @@ public class TestHostFileManager {
             .DatanodeReportType.DEAD).size());
     DatanodeDescriptor spam = new DatanodeDescriptor(new DatanodeID("127.0.0" +
             ".3", "127.0.0.3", "uuid-spam", 12345, 1020, 1021, 1022));
-    spam.setLastUpdate(0);
+    DFSTestUtil.setDatanodeDead(spam);
     includedNodes.add(entry("127.0.0.3:12345"));
     dnMap.put("uuid-spam", spam);
     Assert.assertEquals(1, dm.getDatanodeListForReport(HdfsConstants

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 f797381..c3726f2 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
@@ -137,7 +137,7 @@ public class TestNodeCount {
   
   void initializeTimeout(long timeout) {
     this.timeout = timeout;
-    this.failtime = Time.now()
+    this.failtime = Time.monotonicNow()
         + ((timeout <= 0) ? Long.MAX_VALUE : timeout);
   }
   
@@ -148,7 +148,7 @@ public class TestNodeCount {
   
   /* check for timeout, then wait for cycleTime msec */
   void checkTimeout(String testLabel, long cycleTime) throws TimeoutException {
-    if (Time.now() > failtime) {
+    if (Time.monotonicNow() > failtime) {
       throw new TimeoutException("Timeout: "
           + testLabel + " for block " + lastBlock + " after " + timeout 
           + " msec.  Last counts: live = " + lastNum.liveReplicas()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 6bbb0c3..a86b573 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
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
+import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 public class TestOverReplicatedBlocks {
@@ -171,10 +172,10 @@ public class TestOverReplicatedBlocks {
       long waitTime = DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT * 1000 *
         (DFSConfigKeys.DFS_NAMENODE_TOLERATE_HEARTBEAT_MULTIPLIER_DEFAULT + 1);
       do {
-        nodeInfo = 
-          namesystem.getBlockManager().getDatanodeManager().getDatanode(dnReg);
-        lastHeartbeat = nodeInfo.getLastUpdate();
-      } while(now() - lastHeartbeat < waitTime);
+        nodeInfo = namesystem.getBlockManager().getDatanodeManager()
+            .getDatanode(dnReg);
+        lastHeartbeat = nodeInfo.getLastUpdateMonotonic();
+      } while (monotonicNow() - lastHeartbeat < waitTime);
       fs.setReplication(fileName, (short)3);
 
       BlockLocation locs[] = fs.getFileBlockLocations(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 24fd81d..485cb9b 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
@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.LogVerificationAppender;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.TestBlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager.StatefulBlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -564,7 +565,7 @@ public class TestReplicationPolicy {
   @Test
   public void testChooseTargetWithStaleNodes() throws Exception {
     // Set dataNodes[0] as stale
-    dataNodes[0].setLastUpdate(Time.now() - staleInterval - 1);
+    DFSTestUtil.resetLastUpdatesWithOffset(dataNodes[0], -(staleInterval + 1));
     namenode.getNamesystem().getBlockManager()
       .getDatanodeManager().getHeartbeatManager().heartbeatCheck();
     assertTrue(namenode.getNamesystem().getBlockManager()
@@ -584,7 +585,7 @@ public class TestReplicationPolicy {
     assertFalse(isOnSameRack(targets[0], dataNodes[0]));
     
     // reset
-    dataNodes[0].setLastUpdate(Time.now());
+    DFSTestUtil.resetLastUpdatesWithOffset(dataNodes[0], 0);
     namenode.getNamesystem().getBlockManager()
       .getDatanodeManager().getHeartbeatManager().heartbeatCheck();
   }
@@ -601,7 +602,8 @@ public class TestReplicationPolicy {
   public void testChooseTargetWithHalfStaleNodes() throws Exception {
     // Set dataNodes[0], dataNodes[1], and dataNodes[2] as stale
     for (int i = 0; i < 3; i++) {
-      dataNodes[i].setLastUpdate(Time.now() - staleInterval - 1);
+      DFSTestUtil
+          .resetLastUpdatesWithOffset(dataNodes[i], -(staleInterval + 1));
     }
     namenode.getNamesystem().getBlockManager()
       .getDatanodeManager().getHeartbeatManager().heartbeatCheck();
@@ -633,7 +635,7 @@ public class TestReplicationPolicy {
     assertTrue(containsWithinRange(dataNodes[5], targets, 0, 3));
 
     for (int i = 0; i < dataNodes.length; i++) {
-      dataNodes[i].setLastUpdate(Time.now());
+      DFSTestUtil.resetLastUpdatesWithOffset(dataNodes[i], 0);
     }
     namenode.getNamesystem().getBlockManager()
       .getDatanodeManager().getHeartbeatManager().heartbeatCheck();
@@ -659,9 +661,10 @@ public class TestReplicationPolicy {
       for (int i = 0; i < 2; i++) {
         DataNode dn = miniCluster.getDataNodes().get(i);
         DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
-        miniCluster.getNameNode().getNamesystem().getBlockManager()
-            .getDatanodeManager().getDatanode(dn.getDatanodeId())
-            .setLastUpdate(Time.now() - staleInterval - 1);
+        DatanodeDescriptor dnDes = miniCluster.getNameNode().getNamesystem()
+            .getBlockManager().getDatanodeManager()
+            .getDatanode(dn.getDatanodeId());
+        DFSTestUtil.resetLastUpdatesWithOffset(dnDes, -(staleInterval + 1));
       }
       // Instead of waiting, explicitly call heartbeatCheck to 
       // let heartbeat manager to detect stale nodes
@@ -689,9 +692,9 @@ public class TestReplicationPolicy {
       for (int i = 0; i < 4; i++) {
         DataNode dn = miniCluster.getDataNodes().get(i);
         DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true);
-        miniCluster.getNameNode().getNamesystem().getBlockManager()
-            .getDatanodeManager().getDatanode(dn.getDatanodeId())
-            .setLastUpdate(Time.now() - staleInterval - 1);
+        DatanodeDescriptor dnDesc = miniCluster.getNameNode().getNamesystem().getBlockManager()
+            .getDatanodeManager().getDatanode(dn.getDatanodeId());
+        DFSTestUtil.resetLastUpdatesWithOffset(dnDesc, -(staleInterval + 1));
       }
       // Explicitly call heartbeatCheck
       miniCluster.getNameNode().getNamesystem().getBlockManager()
@@ -710,14 +713,15 @@ public class TestReplicationPolicy {
       assertEquals(targets.length, 3);
       assertTrue(isOnSameRack(targets[0], staleNodeInfo));
       
-      // Step 3. Set 2 stale datanodes back to healthy nodes, 
+      // Step 3. Set 2 stale datanodes back to healthy nodes,
       // still have 2 stale nodes
       for (int i = 2; i < 4; i++) {
         DataNode dn = miniCluster.getDataNodes().get(i);
         DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false);
-        miniCluster.getNameNode().getNamesystem().getBlockManager()
-            .getDatanodeManager().getDatanode(dn.getDatanodeId())
-            .setLastUpdate(Time.now());
+        DatanodeDescriptor dnDesc = miniCluster.getNameNode().getNamesystem()
+            .getBlockManager().getDatanodeManager()
+            .getDatanode(dn.getDatanodeId());
+        DFSTestUtil.resetLastUpdatesWithOffset(dnDesc, 0);
       }
       // Explicitly call heartbeatCheck
       miniCluster.getNameNode().getNamesystem().getBlockManager()
@@ -973,7 +977,7 @@ public class TestReplicationPolicy {
     
     // Refresh the last update time for all the datanodes
     for (int i = 0; i < dataNodes.length; i++) {
-      dataNodes[i].setLastUpdate(Time.now());
+      DFSTestUtil.resetLastUpdatesWithOffset(dataNodes[i], 0);
     }
     
     List<DatanodeStorageInfo> first = new ArrayList<DatanodeStorageInfo>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
index 37c503c..de66db5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
@@ -660,12 +660,12 @@ public abstract class BlockReportTestBase {
     final DataNode dn1 = cluster.getDataNodes().get(DN_N1);
     String bpid = cluster.getNamesystem().getBlockPoolId();
     Replica r = DataNodeTestUtils.fetchReplicaInfo(dn1, bpid, bl.getBlockId());
-    long start = Time.now();
+    long start = Time.monotonicNow();
     int count = 0;
     while (r == null) {
       waitTil(5);
       r = DataNodeTestUtils.fetchReplicaInfo(dn1, bpid, bl.getBlockId());
-      long waiting_period = Time.now() - start;
+      long waiting_period = Time.monotonicNow() - start;
       if (count++ % 100 == 0)
         if(LOG.isDebugEnabled()) {
           LOG.debug("Has been waiting for " + waiting_period + " ms.");
@@ -679,7 +679,7 @@ public abstract class BlockReportTestBase {
     if(LOG.isDebugEnabled()) {
       LOG.debug("Replica state before the loop " + state.getValue());
     }
-    start = Time.now();
+    start = Time.monotonicNow();
     while (state != HdfsServerConstants.ReplicaState.TEMPORARY) {
       waitTil(5);
       state = r.getState();
@@ -687,7 +687,7 @@ public abstract class BlockReportTestBase {
         LOG.debug("Keep waiting for " + bl.getBlockName() +
             " is in state " + state.getValue());
       }
-      if (Time.now() - start > TIMEOUT)
+      if (Time.monotonicNow() - start > TIMEOUT)
         assertTrue("Was waiting too long for a replica to become TEMPORARY",
           tooLongWait);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
index 1f2c1b7..2c4795a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockReplacement.java
@@ -73,7 +73,7 @@ public class TestBlockReplacement {
     long bandwidthPerSec = 1024*1024L;
     final long TOTAL_BYTES =6*bandwidthPerSec; 
     long bytesToSend = TOTAL_BYTES; 
-    long start = Time.now();
+    long start = Time.monotonicNow();
     DataTransferThrottler throttler = new DataTransferThrottler(bandwidthPerSec);
     long totalBytes = 0L;
     long bytesSent = 1024*512L; // 0.5MB
@@ -86,7 +86,7 @@ public class TestBlockReplacement {
       Thread.sleep(1000);
     } catch (InterruptedException ignored) {}
     throttler.throttle(bytesToSend);
-    long end = Time.now();
+    long end = Time.monotonicNow();
     assertTrue(totalBytes*1000/(end-start)<=bandwidthPerSec);
   }
   
@@ -254,7 +254,7 @@ public class TestBlockReplacement {
       throws IOException, TimeoutException {
     boolean notDone;
     final long TIMEOUT = 20000L;
-    long starttime = Time.now();
+    long starttime = Time.monotonicNow();
     long failtime = starttime + TIMEOUT;
     do {
       try {
@@ -279,7 +279,7 @@ public class TestBlockReplacement {
           }
         }
       }
-      if (Time.now() > failtime) {
+      if (Time.monotonicNow() > failtime) {
         String expectedNodesList = "";
         String currentNodesList = "";
         for (DatanodeInfo dn : includeNodes) 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
index 35a611b..fd611ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNamenodeCapacityReport.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.DF;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSOutputStream;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
@@ -199,7 +200,7 @@ public class TestNamenodeCapacityReport {
         DataNode dn = datanodes.get(i);
         DatanodeDescriptor dnd = dnm.getDatanode(dn.getDatanodeId());
         dn.shutdown();
-        dnd.setLastUpdate(0L);
+        DFSTestUtil.setDatanodeDead(dnd);
         BlockManagerTestUtil.checkHeartbeat(namesystem.getBlockManager());
         expectedInServiceNodes--;
         assertEquals(expectedInServiceNodes, namesystem.getNumLiveDataNodes());
@@ -278,7 +279,7 @@ public class TestNamenodeCapacityReport {
         dn.shutdown();
         // force it to appear dead so live count decreases
         DatanodeDescriptor dnDesc = dnm.getDatanode(dn.getDatanodeId());
-        dnDesc.setLastUpdate(0L);
+        DFSTestUtil.setDatanodeDead(dnDesc);
         BlockManagerTestUtil.checkHeartbeat(namesystem.getBlockManager());
         assertEquals(nodes-1-i, namesystem.getNumLiveDataNodes());
         // first few nodes are already out of service

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
index 6771ad8..9e96a8f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/metrics/TestNameNodeMetrics.java
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -162,9 +163,10 @@ public class TestNameNodeMetrics {
       long staleInterval = CONF.getLong(
           DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY,
           DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_DEFAULT);
-      cluster.getNameNode().getNamesystem().getBlockManager()
-          .getDatanodeManager().getDatanode(dn.getDatanodeId())
-          .setLastUpdate(Time.now() - staleInterval - 1);
+      DatanodeDescriptor dnDes = cluster.getNameNode().getNamesystem()
+          .getBlockManager().getDatanodeManager()
+          .getDatanode(dn.getDatanodeId());
+      DFSTestUtil.resetLastUpdatesWithOffset(dnDes, -(staleInterval + 1));
     }
     // Let HeartbeatManager to check heartbeat
     BlockManagerTestUtil.checkHeartbeat(cluster.getNameNode().getNamesystem()
@@ -175,9 +177,10 @@ public class TestNameNodeMetrics {
     for (int i = 0; i < 2; i++) {
       DataNode dn = cluster.getDataNodes().get(i);
       DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false);
-      cluster.getNameNode().getNamesystem().getBlockManager()
-          .getDatanodeManager().getDatanode(dn.getDatanodeId())
-          .setLastUpdate(Time.now());
+      DatanodeDescriptor dnDes = cluster.getNameNode().getNamesystem()
+          .getBlockManager().getDatanodeManager()
+          .getDatanode(dn.getDatanodeId());
+      DFSTestUtil.resetLastUpdatesWithOffset(dnDes, 0);
     }
     
     // Let HeartbeatManager to refresh


[35/51] [abbrv] hadoop git commit: HDFS-7961. Trigger full block report after hot swapping disk. Contributed by Eddy Xu.

Posted by ka...@apache.org.
HDFS-7961. Trigger full block report after hot swapping disk. Contributed by Eddy Xu.


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

Branch: refs/heads/YARN-2139
Commit: 6413d34986f3399023426c89c9a0d401c9557716
Parents: 4170c99
Author: Andrew Wang <wa...@apache.org>
Authored: Tue Mar 24 09:07:02 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Tue Mar 24 09:07:02 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 +
 .../hadoop/hdfs/server/datanode/DataNode.java   |  4 ++
 .../datanode/TestDataNodeHotSwapVolumes.java    | 42 ++++++++++++++++++++
 3 files changed, 48 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6413d349/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 ee9a5db..70be18a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1251,6 +1251,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-7960. The full block report should prune zombie storages even if
     they're not empty. (cmccabe and Eddy Xu via wang)
 
+    HDFS-7961. Trigger full block report after hot swapping disk. (Eddy Xu via wang)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6413d349/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 e9befb4..d94375e 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
@@ -634,6 +634,10 @@ public class DataNode extends ReconfigurableBase
       conf.set(DFS_DATANODE_DATA_DIR_KEY,
           Joiner.on(",").join(effectiveVolumes));
       dataDirs = getStorageLocations(conf);
+
+      // Send a full block report to let NN acknowledge the volume changes.
+      triggerBlockReport(new BlockReportOptions.Factory()
+          .setIncremental(false).build());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6413d349/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
index 2f51d45..f5772e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
@@ -34,12 +34,16 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
+import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.junit.After;
 import org.junit.Test;
@@ -59,6 +63,7 @@ import java.util.concurrent.TimeoutException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.mockito.Mockito;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
 import static org.hamcrest.CoreMatchers.anyOf;
@@ -70,6 +75,9 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.timeout;
 
 public class TestDataNodeHotSwapVolumes {
   private static final Log LOG = LogFactory.getLog(
@@ -702,4 +710,38 @@ public class TestDataNodeHotSwapVolumes {
     // More data has been written to this volume.
     assertTrue(restoredVolume.getDfsUsed() > used);
   }
+
+  /** Test that a full block report is sent after hot swapping volumes */
+  @Test(timeout=100000)
+  public void testFullBlockReportAfterRemovingVolumes()
+      throws IOException, ReconfigurationException {
+
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+
+    // Similar to TestTriggerBlockReport, set a really long value for
+    // dfs.heartbeat.interval, so that incremental block reports and heartbeats
+    // won't be sent during this test unless they're triggered
+    // manually.
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10800000L);
+    conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1080L);
+
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
+    cluster.waitActive();
+
+    final DataNode dn = cluster.getDataNodes().get(0);
+    DatanodeProtocolClientSideTranslatorPB spy =
+        DataNodeTestUtils.spyOnBposToNN(dn, cluster.getNameNode());
+
+    // Remove a data dir from datanode
+    File dataDirToKeep = new File(cluster.getDataDirectory(), "data1");
+    dn.reconfigurePropertyImpl(DFS_DATANODE_DATA_DIR_KEY, dataDirToKeep.toString());
+
+    // We should get 1 full report
+    Mockito.verify(spy, timeout(60000).times(1)).blockReport(
+        any(DatanodeRegistration.class),
+        anyString(),
+        any(StorageBlockReport[].class),
+        any(BlockReportContext.class));
+  }
 }


[43/51] [abbrv] hadoop git commit: HDFS-6826. Plugin interface to enable delegation of HDFS authorization assertions. Contributed by Arun Suresh.

Posted by ka...@apache.org.
HDFS-6826. Plugin interface to enable delegation of HDFS authorization assertions. Contributed by Arun Suresh.


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

Branch: refs/heads/YARN-2139
Commit: 53a28afe293e5bf185c8d4f2c7aea212e66015c2
Parents: e38ef70
Author: Jitendra Pandey <Ji...@Jitendra-Pandeys-MacBook-Pro-4.local>
Authored: Tue Mar 24 15:43:03 2015 -0700
Committer: Jitendra Pandey <Ji...@Jitendra-Pandeys-MacBook-Pro-4.local>
Committed: Tue Mar 24 16:02:40 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   1 +
 .../DefaultINodeAttributesProvider.java         |  45 ++++
 .../server/namenode/FSDirStatAndListingOp.java  |  51 +++--
 .../hdfs/server/namenode/FSDirectory.java       |  41 +++-
 .../hdfs/server/namenode/FSEditLogLoader.java   |   6 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  19 ++
 .../server/namenode/FSPermissionChecker.java    | 222 +++++++++++-------
 .../server/namenode/INodeAttributeProvider.java | 135 +++++++++++
 .../hdfs/server/namenode/INodeAttributes.java   |   3 +
 .../namenode/INodeDirectoryAttributes.java      |   4 +
 .../server/namenode/INodeFileAttributes.java    |   5 +
 .../hdfs/server/namenode/INodesInPath.java      |   6 +
 .../namenode/TestFSPermissionChecker.java       |   4 +-
 .../namenode/TestINodeAttributeProvider.java    | 229 +++++++++++++++++++
 15 files changed, 659 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/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 5ade5fb..4bed2ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -390,6 +390,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7838. Expose truncate API for libhdfs. (yliu)
 
+    HDFS-6826. Plugin interface to enable delegation of HDFS authorization 
+    assertions. (Arun Suresh via jitendra)
+
   IMPROVEMENTS
 
     HDFS-7752. Improve description for

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/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 9ecf242..b5bbe5f 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
@@ -477,6 +477,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_IPC_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_DATANODE_IPC_DEFAULT_PORT;
   public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY = "dfs.datanode.min.supported.namenode.version";
   public static final String  DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT = "3.0.0-SNAPSHOT";
+  public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY = "dfs.namenode.inode.attributes.provider.class";
 
   public static final String  DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY = "dfs.block.access.token.enable";
   public static final boolean DFS_BLOCK_ACCESS_TOKEN_ENABLE_DEFAULT = false;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DefaultINodeAttributesProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DefaultINodeAttributesProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DefaultINodeAttributesProvider.java
new file mode 100644
index 0000000..45aa1b5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/DefaultINodeAttributesProvider.java
@@ -0,0 +1,45 @@
+/**
+ * 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;
+
+/**
+ * A default implementation of the INodeAttributesProvider
+ *
+ */
+public class DefaultINodeAttributesProvider extends INodeAttributeProvider {
+
+  public static INodeAttributeProvider DEFAULT_PROVIDER =
+      new DefaultINodeAttributesProvider();
+
+  @Override
+  public void start() {
+    // NO-OP
+  }
+
+  @Override
+  public void stop() {
+    // NO-OP
+  }
+
+  @Override
+  public INodeAttributes getAttributes(String[] pathElements,
+      INodeAttributes inode) {
+    return inode;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/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 cb3da19..43c2de3 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
@@ -181,7 +181,7 @@ class FSDirStatAndListingOp {
 
       if (!targetNode.isDirectory()) {
         return new DirectoryListing(
-            new HdfsFileStatus[]{createFileStatus(fsd,
+            new HdfsFileStatus[]{createFileStatus(fsd, src,
                 HdfsFileStatus.EMPTY_NAME, targetNode, needLocation,
                 parentStoragePolicy, snapshot, isRawPath, iip)}, 0);
       }
@@ -200,7 +200,7 @@ class FSDirStatAndListingOp {
         byte curPolicy = isSuperUser && !cur.isSymlink()?
             cur.getLocalStoragePolicyID():
             BlockStoragePolicySuite.ID_UNSPECIFIED;
-        listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(), cur,
+        listing[i] = createFileStatus(fsd, src, cur.getLocalNameBytes(), cur,
             needLocation, getStoragePolicyID(curPolicy,
                 parentStoragePolicy), snapshot, isRawPath, iip);
         listingCnt++;
@@ -253,7 +253,7 @@ class FSDirStatAndListingOp {
     final HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
     for (int i = 0; i < numOfListing; i++) {
       Snapshot.Root sRoot = snapshots.get(i + skipSize).getRoot();
-      listing[i] = createFileStatus(fsd, sRoot.getLocalNameBytes(), sRoot,
+      listing[i] = createFileStatus(fsd, src, sRoot.getLocalNameBytes(), sRoot,
           BlockStoragePolicySuite.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
           false, INodesInPath.fromINode(sRoot));
     }
@@ -270,7 +270,7 @@ class FSDirStatAndListingOp {
    *         or null if file not found
    */
   static HdfsFileStatus getFileInfo(
-      FSDirectory fsd, INodesInPath src, boolean isRawPath,
+      FSDirectory fsd, String path, INodesInPath src, boolean isRawPath,
       boolean includeStoragePolicy)
       throws IOException {
     fsd.readLock();
@@ -279,7 +279,7 @@ class FSDirStatAndListingOp {
       byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
           i.getStoragePolicyID() : BlockStoragePolicySuite.ID_UNSPECIFIED;
       return i == null ? null : createFileStatus(
-          fsd, HdfsFileStatus.EMPTY_NAME, i, policyId,
+          fsd, path, HdfsFileStatus.EMPTY_NAME, i, policyId,
           src.getPathSnapshotId(), isRawPath, src);
     } finally {
       fsd.readUnlock();
@@ -303,7 +303,7 @@ class FSDirStatAndListingOp {
     fsd.readLock();
     try {
       final INodesInPath iip = fsd.getINodesInPath(srcs, resolveLink);
-      return getFileInfo(fsd, iip, isRawPath, includeStoragePolicy);
+      return getFileInfo(fsd, src, iip, isRawPath, includeStoragePolicy);
     } finally {
       fsd.readUnlock();
     }
@@ -340,14 +340,15 @@ class FSDirStatAndListingOp {
    * @throws java.io.IOException if any error occurs
    */
   static HdfsFileStatus createFileStatus(
-      FSDirectory fsd, byte[] path, INode node, boolean needLocation,
-      byte storagePolicy, int snapshot, boolean isRawPath, INodesInPath iip)
+      FSDirectory fsd, String fullPath, byte[] path, INode node,
+      boolean needLocation, byte storagePolicy, int snapshot, boolean isRawPath,
+      INodesInPath iip)
       throws IOException {
     if (needLocation) {
-      return createLocatedFileStatus(fsd, path, node, storagePolicy,
+      return createLocatedFileStatus(fsd, fullPath, path, node, storagePolicy,
           snapshot, isRawPath, iip);
     } else {
-      return createFileStatus(fsd, path, node, storagePolicy, snapshot,
+      return createFileStatus(fsd, fullPath, path, node, storagePolicy, snapshot,
           isRawPath, iip);
     }
   }
@@ -356,8 +357,9 @@ class FSDirStatAndListingOp {
    * Create FileStatus by file INode
    */
   static HdfsFileStatus createFileStatus(
-      FSDirectory fsd, byte[] path, INode node, byte storagePolicy,
-      int snapshot, boolean isRawPath, INodesInPath iip) throws IOException {
+      FSDirectory fsd, String fullPath, byte[] path, INode node,
+      byte storagePolicy, int snapshot, boolean isRawPath,
+      INodesInPath iip) throws IOException {
      long size = 0;     // length is zero for directories
      short replication = 0;
      long blocksize = 0;
@@ -380,6 +382,8 @@ class FSDirStatAndListingOp {
      int childrenNum = node.isDirectory() ?
          node.asDirectory().getChildrenNum(snapshot) : 0;
 
+     INodeAttributes nodeAttrs =
+         fsd.getAttributes(fullPath, path, node, snapshot);
      return new HdfsFileStatus(
         size,
         node.isDirectory(),
@@ -387,9 +391,9 @@ class FSDirStatAndListingOp {
         blocksize,
         node.getModificationTime(snapshot),
         node.getAccessTime(snapshot),
-        getPermissionForFileStatus(node, snapshot, isEncrypted),
-        node.getUserName(snapshot),
-        node.getGroupName(snapshot),
+        getPermissionForFileStatus(nodeAttrs, isEncrypted),
+        nodeAttrs.getUserName(),
+        nodeAttrs.getGroupName(),
         node.isSymlink() ? node.asSymlink().getSymlink() : null,
         path,
         node.getId(),
@@ -402,8 +406,9 @@ class FSDirStatAndListingOp {
    * Create FileStatus with location info by file INode
    */
   private static HdfsLocatedFileStatus createLocatedFileStatus(
-      FSDirectory fsd, byte[] path, INode node, byte storagePolicy,
-      int snapshot, boolean isRawPath, INodesInPath iip) throws IOException {
+      FSDirectory fsd, String fullPath, byte[] path, INode node,
+      byte storagePolicy, int snapshot, boolean isRawPath,
+      INodesInPath iip) throws IOException {
     assert fsd.hasReadLock();
     long size = 0; // length is zero for directories
     short replication = 0;
@@ -437,12 +442,14 @@ class FSDirStatAndListingOp {
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
 
+    INodeAttributes nodeAttrs =
+        fsd.getAttributes(fullPath, path, node, snapshot);
     HdfsLocatedFileStatus status =
         new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
           blocksize, node.getModificationTime(snapshot),
           node.getAccessTime(snapshot),
-          getPermissionForFileStatus(node, snapshot, isEncrypted),
-          node.getUserName(snapshot), node.getGroupName(snapshot),
+          getPermissionForFileStatus(nodeAttrs, isEncrypted),
+          nodeAttrs.getUserName(), nodeAttrs.getGroupName(),
           node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
           node.getId(), loc, childrenNum, feInfo, storagePolicy);
     // Set caching information for the located blocks.
@@ -467,9 +474,9 @@ class FSDirStatAndListingOp {
    * and encrypted bit on if it represents an encrypted file/dir.
    */
   private static FsPermission getPermissionForFileStatus(
-      INode node, int snapshot, boolean isEncrypted) {
-    FsPermission perm = node.getFsPermission(snapshot);
-    boolean hasAcl = node.getAclFeature(snapshot) != null;
+      INodeAttributes node, boolean isEncrypted) {
+    FsPermission perm = node.getFsPermission();
+    boolean hasAcl = node.getAclFeature() != null;
     if (hasAcl || isEncrypted) {
       perm = new FsPermissionExtension(perm, hasAcl, isEncrypted);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/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 2f73627..7eea343 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
@@ -167,6 +167,12 @@ public class FSDirectory implements Closeable {
 
   private final FSEditLog editLog;
 
+  private INodeAttributeProvider attributeProvider;
+
+  public void setINodeAttributeProvider(INodeAttributeProvider provider) {
+    attributeProvider = provider;
+  }
+
   // utility methods to acquire and release read lock and write lock
   void readLock() {
     this.dirLock.readLock().lock();
@@ -1623,13 +1629,23 @@ public class FSDirectory implements Closeable {
   FSPermissionChecker getPermissionChecker()
     throws AccessControlException {
     try {
-      return new FSPermissionChecker(fsOwnerShortUserName, supergroup,
+      return getPermissionChecker(fsOwnerShortUserName, supergroup,
           NameNode.getRemoteUser());
-    } catch (IOException ioe) {
-      throw new AccessControlException(ioe);
+    } catch (IOException e) {
+      throw new AccessControlException(e);
     }
   }
 
+  @VisibleForTesting
+  FSPermissionChecker getPermissionChecker(String fsOwner, String superGroup,
+      UserGroupInformation ugi) throws AccessControlException {
+    return new FSPermissionChecker(
+        fsOwner, superGroup, ugi,
+        attributeProvider == null ?
+            DefaultINodeAttributesProvider.DEFAULT_PROVIDER
+            : attributeProvider);
+  }
+
   void checkOwner(FSPermissionChecker pc, INodesInPath iip)
       throws AccessControlException {
     checkPermission(pc, iip, true, null, null, null, null);
@@ -1690,7 +1706,8 @@ public class FSDirectory implements Closeable {
   HdfsFileStatus getAuditFileInfo(INodesInPath iip)
       throws IOException {
     return (namesystem.isAuditEnabled() && namesystem.isExternalInvocation())
-        ? FSDirStatAndListingOp.getFileInfo(this, iip, false, false) : null;
+        ? FSDirStatAndListingOp.getFileInfo(this, iip.getPath(), iip, false,
+            false) : null;
   }
 
   /**
@@ -1736,4 +1753,20 @@ public class FSDirectory implements Closeable {
   void resetLastInodeIdWithoutChecking(long newValue) {
     inodeId.setCurrentValue(newValue);
   }
+
+  INodeAttributes getAttributes(String fullPath, byte[] path,
+      INode node, int snapshot) {
+    INodeAttributes nodeAttrs = node;
+    if (attributeProvider != null) {
+      nodeAttrs = node.getSnapshotINode(snapshot);
+      fullPath = fullPath + (fullPath.endsWith(Path.SEPARATOR) ? ""
+                                                               : Path.SEPARATOR)
+          + DFSUtil.bytes2String(path);
+      nodeAttrs = attributeProvider.getAttributes(fullPath, nodeAttrs);
+    } else {
+      nodeAttrs = node.getSnapshotINode(snapshot);
+    }
+    return nodeAttrs;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/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 ad661ca..f50dc4d 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
@@ -378,7 +378,7 @@ public class FSEditLogLoader {
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
           HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
-              fsNamesys.dir, HdfsFileStatus.EMPTY_NAME, newFile,
+              fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, newFile,
               BlockStoragePolicySuite.ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
               false, iip);
           fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
@@ -397,7 +397,7 @@ public class FSEditLogLoader {
           // add the op into retry cache if necessary
           if (toAddRetryCache) {
             HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
-                fsNamesys.dir,
+                fsNamesys.dir, path,
                 HdfsFileStatus.EMPTY_NAME, newFile,
                 BlockStoragePolicySuite.ID_UNSPECIFIED,
                 Snapshot.CURRENT_STATE_ID, false, iip);
@@ -471,7 +471,7 @@ public class FSEditLogLoader {
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
           HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatus(
-              fsNamesys.dir, HdfsFileStatus.EMPTY_NAME, file,
+              fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME, file,
               BlockStoragePolicySuite.ID_UNSPECIFIED,
               Snapshot.CURRENT_STATE_ID, false, iip);
           fsNamesys.addCacheEntryWithPayload(appendOp.rpcClientId,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/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 34b5e95..9235425 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
@@ -62,6 +62,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CAC
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
@@ -277,6 +278,7 @@ import org.apache.hadoop.security.token.delegation.DelegationKey;
 import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.log4j.Appender;
@@ -536,6 +538,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   private final TopConf topConf;
   private TopMetrics topMetrics;
 
+  private INodeAttributeProvider inodeAttributeProvider;
+
   /**
    * Notify that loading of this FSDirectory is complete, and
    * it is imageLoaded for use
@@ -841,6 +845,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       this.isDefaultAuditLogger = auditLoggers.size() == 1 &&
         auditLoggers.get(0) instanceof DefaultAuditLogger;
       this.retryCache = ignoreRetryCache ? null : initRetryCache(conf);
+      Class<? extends INodeAttributeProvider> klass = conf.getClass(
+          DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY,
+          null, INodeAttributeProvider.class);
+      if (klass != null) {
+        inodeAttributeProvider = ReflectionUtils.newInstance(klass, conf);
+        LOG.info("Using INode attribute provider: " + klass.getName());
+      }
     } catch(IOException e) {
       LOG.error(getClass().getSimpleName() + " initialization failed.", e);
       close();
@@ -1067,6 +1078,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     
     registerMXBean();
     DefaultMetricsSystem.instance().register(this);
+    if (inodeAttributeProvider != null) {
+      inodeAttributeProvider.start();
+      dir.setINodeAttributeProvider(inodeAttributeProvider);
+    }
     snapshotManager.registerMXBean();
   }
   
@@ -1075,6 +1090,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   void stopCommonServices() {
     writeLock();
+    if (inodeAttributeProvider != null) {
+      dir.setINodeAttributeProvider(null);
+      inodeAttributeProvider.stop();
+    }
     try {
       if (blockManager != null) blockManager.close();
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
index 0508484..e6570f5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSPermissionChecker.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.HashSet;
-import java.util.List;
 import java.util.Set;
 import java.util.Stack;
 
@@ -30,6 +29,8 @@ import org.apache.hadoop.fs.permission.AclEntryScope;
 import org.apache.hadoop.fs.permission.AclEntryType;
 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.server.namenode.INodeAttributeProvider.AccessControlEnforcer;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -41,25 +42,25 @@ import org.apache.hadoop.security.UserGroupInformation;
  * 
  * Some of the helper methods are gaurded by {@link FSNamesystem#readLock()}.
  */
-class FSPermissionChecker {
+class FSPermissionChecker implements AccessControlEnforcer {
   static final Log LOG = LogFactory.getLog(UserGroupInformation.class);
 
   /** @return a string for throwing {@link AccessControlException} */
-  private String toAccessControlString(INode inode, int snapshotId,
+  private String toAccessControlString(INodeAttributes inodeAttrib, String path,
       FsAction access, FsPermission mode) {
-    return toAccessControlString(inode, snapshotId, access, mode, false);
+    return toAccessControlString(inodeAttrib, path, access, mode, false);
   }
 
   /** @return a string for throwing {@link AccessControlException} */
-  private String toAccessControlString(INode inode, int snapshotId, FsAction access,
-      FsPermission mode, boolean deniedFromAcl) {
+  private String toAccessControlString(INodeAttributes inodeAttrib,
+      String path, FsAction access, FsPermission mode, boolean deniedFromAcl) {
     StringBuilder sb = new StringBuilder("Permission denied: ")
-      .append("user=").append(user).append(", ")
+      .append("user=").append(getUser()).append(", ")
       .append("access=").append(access).append(", ")
-      .append("inode=\"").append(inode.getFullPathName()).append("\":")
-      .append(inode.getUserName(snapshotId)).append(':')
-      .append(inode.getGroupName(snapshotId)).append(':')
-      .append(inode.isDirectory() ? 'd' : '-')
+      .append("inode=\"").append(path).append("\":")
+      .append(inodeAttrib.getUserName()).append(':')
+      .append(inodeAttrib.getGroupName()).append(':')
+      .append(inodeAttrib.isDirectory() ? 'd' : '-')
       .append(mode);
     if (deniedFromAcl) {
       sb.append("+");
@@ -67,42 +68,59 @@ class FSPermissionChecker {
     return sb.toString();
   }
 
+  private final String fsOwner;
+  private final String supergroup;
+  private final UserGroupInformation callerUgi;
+
   private final String user;
-  /** A set with group namess. Not synchronized since it is unmodifiable */
   private final Set<String> groups;
   private final boolean isSuper;
+  private final INodeAttributeProvider attributeProvider;
+
 
   FSPermissionChecker(String fsOwner, String supergroup,
-      UserGroupInformation callerUgi) {
-    HashSet<String> s = new HashSet<String>(Arrays.asList(callerUgi.getGroupNames()));
+      UserGroupInformation callerUgi,
+      INodeAttributeProvider attributeProvider) {
+    this.fsOwner = fsOwner;
+    this.supergroup = supergroup;
+    this.callerUgi = callerUgi;
+    HashSet<String> s =
+        new HashSet<String>(Arrays.asList(callerUgi.getGroupNames()));
     groups = Collections.unmodifiableSet(s);
     user = callerUgi.getShortUserName();
     isSuper = user.equals(fsOwner) || groups.contains(supergroup);
+    this.attributeProvider = attributeProvider;
   }
 
-  /**
-   * Check if the callers group contains the required values.
-   * @param group group to check
-   */
-  public boolean containsGroup(String group) {return groups.contains(group);}
+  public boolean containsGroup(String group) {
+    return groups.contains(group);
+  }
 
   public String getUser() {
     return user;
   }
-  
+
+  public Set<String> getGroups() {
+    return groups;
+  }
+
   public boolean isSuperUser() {
     return isSuper;
   }
-  
+
+  public INodeAttributeProvider getAttributesProvider() {
+    return attributeProvider;
+  }
+
   /**
    * Verify if the caller has the required permission. This will result into 
    * an exception if the caller is not allowed to access the resource.
    */
   public void checkSuperuserPrivilege()
       throws AccessControlException {
-    if (!isSuper) {
+    if (!isSuperUser()) {
       throw new AccessControlException("Access denied for user " 
-          + user + ". Superuser privilege is required");
+          + getUser() + ". Superuser privilege is required");
     }
   }
   
@@ -154,64 +172,98 @@ class FSPermissionChecker {
     // check if (parentAccess != null) && file exists, then check sb
     // If resolveLink, the check is performed on the link target.
     final int snapshotId = inodesInPath.getPathSnapshotId();
-    final int length = inodesInPath.length();
-    final INode last = length > 0 ? inodesInPath.getLastINode() : null;
-    final INode parent = length > 1 ? inodesInPath.getINode(-2) : null;
+    final INode[] inodes = inodesInPath.getINodesArray();
+    final INodeAttributes[] inodeAttrs = new INodeAttributes[inodes.length];
+    final byte[][] pathByNameArr = new byte[inodes.length][];
+    for (int i = 0; i < inodes.length && inodes[i] != null; i++) {
+      if (inodes[i] != null) {
+        pathByNameArr[i] = inodes[i].getLocalNameBytes();
+        inodeAttrs[i] = getINodeAttrs(pathByNameArr, i, inodes[i], snapshotId);
+      }
+    }
+
+    String path = inodesInPath.getPath();
+    int ancestorIndex = inodes.length - 2;
+
+    AccessControlEnforcer enforcer =
+        getAttributesProvider().getExternalAccessControlEnforcer(this);
+    enforcer.checkPermission(fsOwner, supergroup, callerUgi, inodeAttrs, inodes,
+        pathByNameArr, snapshotId, path, ancestorIndex, doCheckOwner,
+        ancestorAccess, parentAccess, access, subAccess, ignoreEmptyDir);
+  }
 
-    checkTraverse(inodesInPath, snapshotId);
+  @Override
+  public void checkPermission(String fsOwner, String supergroup,
+      UserGroupInformation callerUgi, INodeAttributes[] inodeAttrs,
+      INode[] inodes, byte[][] pathByNameArr, int snapshotId, String path,
+      int ancestorIndex, boolean doCheckOwner, FsAction ancestorAccess,
+      FsAction parentAccess, FsAction access, FsAction subAccess,
+      boolean ignoreEmptyDir)
+      throws AccessControlException {
+    for(; ancestorIndex >= 0 && inodes[ancestorIndex] == null;
+        ancestorIndex--);
+    checkTraverse(inodeAttrs, path, ancestorIndex);
 
+    final INodeAttributes last = inodeAttrs[inodeAttrs.length - 1];
     if (parentAccess != null && parentAccess.implies(FsAction.WRITE)
-        && length > 1 && last != null) {
-      checkStickyBit(parent, last, snapshotId);
+        && inodeAttrs.length > 1 && last != null) {
+      checkStickyBit(inodeAttrs[inodeAttrs.length - 2], last);
     }
-    if (ancestorAccess != null && length > 1) {
-      List<INode> inodes = inodesInPath.getReadOnlyINodes();
-      INode ancestor = null;
-      for (int i = inodes.size() - 2; i >= 0 && (ancestor = inodes.get(i)) ==
-          null; i--);
-      check(ancestor, snapshotId, ancestorAccess);
+    if (ancestorAccess != null && inodeAttrs.length > 1) {
+      check(inodeAttrs, path, ancestorIndex, ancestorAccess);
     }
-    if (parentAccess != null && length > 1 && parent != null) {
-      check(parent, snapshotId, parentAccess);
+    if (parentAccess != null && inodeAttrs.length > 1) {
+      check(inodeAttrs, path, inodeAttrs.length - 2, parentAccess);
     }
     if (access != null) {
-      check(last, snapshotId, access);
+      check(last, path, access);
     }
     if (subAccess != null) {
-      checkSubAccess(last, snapshotId, subAccess, ignoreEmptyDir);
+      INode rawLast = inodes[inodeAttrs.length - 1];
+      checkSubAccess(pathByNameArr, inodeAttrs.length - 1, rawLast,
+          snapshotId, subAccess, ignoreEmptyDir);
     }
     if (doCheckOwner) {
-      checkOwner(last, snapshotId);
+      checkOwner(last);
     }
   }
 
+  private INodeAttributes getINodeAttrs(byte[][] pathByNameArr, int pathIdx,
+      INode inode, int snapshotId) {
+    INodeAttributes inodeAttrs = inode.getSnapshotINode(snapshotId);
+    if (getAttributesProvider() != null) {
+      String[] elements = new String[pathIdx + 1];
+      for (int i = 0; i < elements.length; i++) {
+        elements[i] = DFSUtil.bytes2String(pathByNameArr[i]);
+      }
+      inodeAttrs = getAttributesProvider().getAttributes(elements, inodeAttrs);
+    }
+    return inodeAttrs;
+  }
+
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void checkOwner(INode inode, int snapshotId
+  private void checkOwner(INodeAttributes inode
       ) throws AccessControlException {
-    if (inode != null && user.equals(inode.getUserName(snapshotId))) {
+    if (getUser().equals(inode.getUserName())) {
       return;
     }
     throw new AccessControlException(
             "Permission denied. user="
-            + user + " is not the owner of inode=" + inode);
+            + getUser() + " is not the owner of inode=" + inode);
   }
 
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void checkTraverse(INodesInPath iip, int snapshotId)
-      throws AccessControlException {
-    List<INode> inodes = iip.getReadOnlyINodes();
-    for (int i = 0; i < inodes.size() - 1; i++) {
-      INode inode = inodes.get(i);
-      if (inode == null) {
-        break;
-      }
-      check(inode, snapshotId, FsAction.EXECUTE);
+  private void checkTraverse(INodeAttributes[] inodes, String path, int last
+      ) throws AccessControlException {
+    for(int j = 0; j <= last; j++) {
+      check(inodes[j], path, FsAction.EXECUTE);
     }
   }
 
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void checkSubAccess(INode inode, int snapshotId, FsAction access,
-      boolean ignoreEmptyDir) throws AccessControlException {
+  private void checkSubAccess(byte[][] pathByNameArr, int pathIdx, INode inode,
+      int snapshotId, FsAction access, boolean ignoreEmptyDir)
+      throws AccessControlException {
     if (inode == null || !inode.isDirectory()) {
       return;
     }
@@ -221,7 +273,9 @@ class FSPermissionChecker {
       INodeDirectory d = directories.pop();
       ReadOnlyList<INode> cList = d.getChildrenList(snapshotId);
       if (!(cList.isEmpty() && ignoreEmptyDir)) {
-        check(d, snapshotId, access);
+        //TODO have to figure this out with inodeattribute provider
+        check(getINodeAttrs(pathByNameArr, pathIdx, d, snapshotId),
+            inode.getFullPathName(), access);
       }
 
       for(INode child : cList) {
@@ -233,37 +287,37 @@ class FSPermissionChecker {
   }
 
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void check(INode inode, int snapshotId, FsAction access)
-      throws AccessControlException {
+  private void check(INodeAttributes[] inodes, String path, int i, FsAction access
+      ) throws AccessControlException {
+    check(i >= 0 ? inodes[i] : null, path, access);
+  }
+
+  private void check(INodeAttributes inode, String path, FsAction access
+      ) throws AccessControlException {
     if (inode == null) {
       return;
     }
-    FsPermission mode = inode.getFsPermission(snapshotId);
-    AclFeature aclFeature = inode.getAclFeature(snapshotId);
+    final FsPermission mode = inode.getFsPermission();
+    final AclFeature aclFeature = inode.getAclFeature();
     if (aclFeature != null) {
       // It's possible that the inode has a default ACL but no access ACL.
       int firstEntry = aclFeature.getEntryAt(0);
       if (AclEntryStatusFormat.getScope(firstEntry) == AclEntryScope.ACCESS) {
-        checkAccessAcl(inode, snapshotId, access, mode, aclFeature);
+        checkAccessAcl(inode, path, access, mode, aclFeature);
         return;
       }
     }
-    checkFsPermission(inode, snapshotId, access, mode);
-  }
-
-  private void checkFsPermission(INode inode, int snapshotId, FsAction access,
-      FsPermission mode) throws AccessControlException {
-    if (user.equals(inode.getUserName(snapshotId))) { //user class
+    if (getUser().equals(inode.getUserName())) { //user class
       if (mode.getUserAction().implies(access)) { return; }
     }
-    else if (groups.contains(inode.getGroupName(snapshotId))) { //group class
+    else if (getGroups().contains(inode.getGroupName())) { //group class
       if (mode.getGroupAction().implies(access)) { return; }
     }
     else { //other class
       if (mode.getOtherAction().implies(access)) { return; }
     }
     throw new AccessControlException(
-      toAccessControlString(inode, snapshotId, access, mode));
+        toAccessControlString(inode, path, access, mode));
   }
 
   /**
@@ -282,20 +336,20 @@ class FSPermissionChecker {
    * - The other entry must not have a name.
    * - Default entries may be present, but they are ignored during enforcement.
    *
-   * @param inode INode accessed inode
+   * @param inode INodeAttributes accessed inode
    * @param snapshotId int snapshot ID
    * @param access FsAction requested permission
    * @param mode FsPermission mode from inode
    * @param aclFeature AclFeature of inode
    * @throws AccessControlException if the ACL denies permission
    */
-  private void checkAccessAcl(INode inode, int snapshotId, FsAction access,
-      FsPermission mode, AclFeature aclFeature)
+  private void checkAccessAcl(INodeAttributes inode, String path,
+      FsAction access, FsPermission mode, AclFeature aclFeature)
       throws AccessControlException {
     boolean foundMatch = false;
 
     // Use owner entry from permission bits if user is owner.
-    if (user.equals(inode.getUserName(snapshotId))) {
+    if (getUser().equals(inode.getUserName())) {
       if (mode.getUserAction().implies(access)) {
         return;
       }
@@ -314,7 +368,7 @@ class FSPermissionChecker {
         if (type == AclEntryType.USER) {
           // Use named user entry with mask from permission bits applied if user
           // matches name.
-          if (user.equals(name)) {
+          if (getUser().equals(name)) {
             FsAction masked = AclEntryStatusFormat.getPermission(entry).and(
                 mode.getGroupAction());
             if (masked.implies(access)) {
@@ -328,8 +382,8 @@ class FSPermissionChecker {
           // applied if user is a member and entry grants access.  If user is a
           // member of multiple groups that have entries that grant access, then
           // it doesn't matter which is chosen, so exit early after first match.
-          String group = name == null ? inode.getGroupName(snapshotId) : name;
-          if (groups.contains(group)) {
+          String group = name == null ? inode.getGroupName() : name;
+          if (getGroups().contains(group)) {
             FsAction masked = AclEntryStatusFormat.getPermission(entry).and(
                 mode.getGroupAction());
             if (masked.implies(access)) {
@@ -347,28 +401,28 @@ class FSPermissionChecker {
     }
 
     throw new AccessControlException(
-      toAccessControlString(inode, snapshotId, access, mode, true));
+        toAccessControlString(inode, path, access, mode));
   }
 
   /** Guarded by {@link FSNamesystem#readLock()} */
-  private void checkStickyBit(INode parent, INode inode, int snapshotId
+  private void checkStickyBit(INodeAttributes parent, INodeAttributes inode
       ) throws AccessControlException {
-    if(!parent.getFsPermission(snapshotId).getStickyBit()) {
+    if (!parent.getFsPermission().getStickyBit()) {
       return;
     }
 
     // If this user is the directory owner, return
-    if(parent.getUserName(snapshotId).equals(user)) {
+    if (parent.getUserName().equals(getUser())) {
       return;
     }
 
     // if this user is the file owner, return
-    if(inode.getUserName(snapshotId).equals(user)) {
+    if (inode.getUserName().equals(getUser())) {
       return;
     }
 
     throw new AccessControlException("Permission denied by sticky bit setting:" +
-      " user=" + user + ", inode=" + inode);
+      " user=" + getUser() + ", inode=" + inode);
   }
 
   /**
@@ -384,11 +438,11 @@ class FSPermissionChecker {
     if (isSuperUser()) {
       return;
     }
-    if (user.equals(pool.getOwnerName())
+    if (getUser().equals(pool.getOwnerName())
         && mode.getUserAction().implies(access)) {
       return;
     }
-    if (groups.contains(pool.getGroupName())
+    if (getGroups().contains(pool.getGroupName())
         && mode.getGroupAction().implies(access)) {
       return;
     }
@@ -396,7 +450,7 @@ class FSPermissionChecker {
       return;
     }
     throw new AccessControlException("Permission denied while accessing pool "
-        + pool.getPoolName() + ": user " + user + " does not have "
+        + pool.getPoolName() + ": user " + getUser() + " does not have "
         + access.toString() + " permissions.");
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
new file mode 100644
index 0000000..b12e147
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
@@ -0,0 +1,135 @@
+/**
+ * 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.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public abstract class INodeAttributeProvider {
+
+  /**
+   * The AccessControlEnforcer allows implementations to override the
+   * default File System permission checking logic enforced on a file system
+   * object
+   */
+  public interface AccessControlEnforcer {
+
+    /**
+     * Checks permission on a file system object. Has to throw an Exception
+     * if the filesystem object is not accessessible by the calling Ugi.
+     * @param fsOwner Filesystem owner (The Namenode user)
+     * @param supergroup super user geoup
+     * @param callerUgi UserGroupInformation of the caller
+     * @param inodeAttrs Array of INode attributes for each path element in the
+     *                   the path
+     * @param inodes Array of INodes for each path element in the path
+     * @param pathByNameArr Array of byte arrays of the LocalName
+     * @param snapshotId the snapshotId of the requested path
+     * @param path Path String
+     * @param ancestorIndex Index of ancestor
+     * @param doCheckOwner perform ownership check
+     * @param ancestorAccess The access required by the ancestor of the path.
+     * @param parentAccess The access required by the parent of the path.
+     * @param access The access required by the path.
+     * @param subAccess If path is a directory, It is the access required of
+     *                  the path and all the sub-directories. If path is not a
+     *                  directory, there should ideally be no effect.
+     * @param ignoreEmptyDir Ignore permission checking for empty directory?
+     * @throws AccessControlException
+     */
+    public abstract void checkPermission(String fsOwner, String supergroup,
+        UserGroupInformation callerUgi, INodeAttributes[] inodeAttrs,
+        INode[] inodes, byte[][] pathByNameArr, int snapshotId, String path,
+        int ancestorIndex, boolean doCheckOwner, FsAction ancestorAccess,
+        FsAction parentAccess, FsAction access, FsAction subAccess,
+        boolean ignoreEmptyDir)
+            throws AccessControlException;
+
+  }
+  /**
+   * Initialize the provider. This method is called at NameNode startup
+   * time.
+   */
+  public abstract void start();
+
+  /**
+   * Shutdown the provider. This method is called at NameNode shutdown time.
+   */
+  public abstract void stop();
+
+  @VisibleForTesting
+  String[] getPathElements(String path) {
+    path = path.trim();
+    if (path.charAt(0) != Path.SEPARATOR_CHAR) {
+      throw new IllegalArgumentException("It must be an absolute path: " +
+          path);
+    }
+    int numOfElements = StringUtils.countMatches(path, Path.SEPARATOR);
+    if (path.length() > 1 && path.endsWith(Path.SEPARATOR)) {
+      numOfElements--;
+    }
+    String[] pathElements = new String[numOfElements];
+    int elementIdx = 0;
+    int idx = 0;
+    int found = path.indexOf(Path.SEPARATOR_CHAR, idx);
+    while (found > -1) {
+      if (found > idx) {
+        pathElements[elementIdx++] = path.substring(idx, found);
+      }
+      idx = found + 1;
+      found = path.indexOf(Path.SEPARATOR_CHAR, idx);
+    }
+    if (idx < path.length()) {
+      pathElements[elementIdx] = path.substring(idx);
+    }
+    return pathElements;
+  }
+
+  public INodeAttributes getAttributes(String fullPath, INodeAttributes inode) {
+    return getAttributes(getPathElements(fullPath), inode);
+  }
+
+  public abstract INodeAttributes getAttributes(String[] pathElements,
+      INodeAttributes inode);
+
+  /**
+   * Can be over-ridden by implementations to provide a custom Access Control
+   * Enforcer that can provide an alternate implementation of the
+   * default permission checking logic.
+   * @param defaultEnforcer The Default AccessControlEnforcer
+   * @return The AccessControlEnforcer to use
+   */
+  public AccessControlEnforcer getExternalAccessControlEnforcer(
+      AccessControlEnforcer defaultEnforcer) {
+    return defaultEnforcer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributes.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributes.java
index 0f76b68..7b780c2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributes.java
@@ -28,6 +28,9 @@ import org.apache.hadoop.hdfs.server.namenode.XAttrFeature;
  */
 @InterfaceAudience.Private
 public interface INodeAttributes {
+
+  public boolean isDirectory();
+
   /**
    * @return null if the local name is null;
    *         otherwise, return the local name byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryAttributes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryAttributes.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryAttributes.java
index 956deae..240aa15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryAttributes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeDirectoryAttributes.java
@@ -52,6 +52,10 @@ public interface INodeDirectoryAttributes extends INodeAttributes {
           storageSpace(-1).typeSpaces(-1).build();
     }
 
+    public boolean isDirectory() {
+      return true;
+    }
+
     @Override
     public boolean metadataEquals(INodeDirectoryAttributes other) {
       return other != null

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
index 0f85bab..204c8ac 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java
@@ -60,6 +60,11 @@ public interface INodeFileAttributes extends INodeAttributes {
     }
 
     @Override
+    public boolean isDirectory() {
+      return false;
+    }
+
+    @Override
     public short getFileReplication() {
       return HeaderFormat.getReplication(header);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
index 389b62b..f1892c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
@@ -376,6 +376,12 @@ public class INodesInPath {
     return Collections.unmodifiableList(Arrays.asList(inodes));
   }
 
+  public INode[] getINodesArray() {
+    INode[] retArr = new INode[inodes.length];
+    System.arraycopy(inodes, 0, retArr, 0, inodes.length);
+    return retArr;
+  }
+
   /**
    * @param length number of ancestral INodes in the returned INodesInPath
    *               instance

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
index 883029a..0154a03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSPermissionChecker.java
@@ -403,7 +403,7 @@ public class TestFSPermissionChecker {
   private void assertPermissionGranted(UserGroupInformation user, String path,
       FsAction access) throws IOException {
     INodesInPath iip = dir.getINodesInPath(path, true);
-    new FSPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(iip,
+    dir.getPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(iip,
       false, null, null, access, null, false);
   }
 
@@ -411,7 +411,7 @@ public class TestFSPermissionChecker {
       FsAction access) throws IOException {
     try {
       INodesInPath iip = dir.getINodesInPath(path, true);
-      new FSPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(iip,
+      dir.getPermissionChecker(SUPERUSER, SUPERGROUP, user).checkPermission(iip,
         false, null, null, access, null, false);
       fail("expected AccessControlException for user + " + user + ", path = " +
         path + ", access = " + access);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/53a28afe/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
new file mode 100644
index 0000000..111c67c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeAttributeProvider.java
@@ -0,0 +1,229 @@
+/**
+ * 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 java.security.PrivilegedExceptionAction;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclEntryType;
+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.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.server.namenode.INodeAttributeProvider.AccessControlEnforcer;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestINodeAttributeProvider {
+  private MiniDFSCluster miniDFS;
+  private static final Set<String> CALLED = new HashSet<String>();
+
+  public static class MyAuthorizationProvider extends INodeAttributeProvider {
+
+    public static class MyAccessControlEnforcer implements AccessControlEnforcer {
+
+      @Override
+      public void checkPermission(String fsOwner, String supergroup,
+          UserGroupInformation ugi, INodeAttributes[] inodeAttrs,
+          INode[] inodes, byte[][] pathByNameArr, int snapshotId, String path,
+          int ancestorIndex, boolean doCheckOwner, FsAction ancestorAccess,
+          FsAction parentAccess, FsAction access, FsAction subAccess,
+          boolean ignoreEmptyDir) throws AccessControlException {
+        CALLED.add("checkPermission|" + ancestorAccess + "|" + parentAccess + "|" + access);
+      }
+    }
+
+    @Override
+    public void start() {
+      CALLED.add("start");
+    }
+
+    @Override
+    public void stop() {
+      CALLED.add("stop");
+    }
+
+    @Override
+    public INodeAttributes getAttributes(String[] pathElements,
+        final INodeAttributes inode) {
+      CALLED.add("getAttributes");
+      final boolean useDefault = useDefault(pathElements);
+      return new INodeAttributes() {
+        @Override
+        public boolean isDirectory() {
+          return inode.isDirectory();
+        }
+
+        @Override
+        public byte[] getLocalNameBytes() {
+          return inode.getLocalNameBytes();
+        }
+
+        @Override
+        public String getUserName() {
+          return (useDefault) ? inode.getUserName() : "foo";
+        }
+
+        @Override
+        public String getGroupName() {
+          return (useDefault) ? inode.getGroupName() : "bar";
+        }
+
+        @Override
+        public FsPermission getFsPermission() {
+          return (useDefault) ? inode.getFsPermission()
+                              : new FsPermission(getFsPermissionShort());
+        }
+
+        @Override
+        public short getFsPermissionShort() {
+          return (useDefault) ? inode.getFsPermissionShort()
+                              : (short) getPermissionLong();
+        }
+
+        @Override
+        public long getPermissionLong() {
+          return (useDefault) ? inode.getPermissionLong() : 0770;
+        }
+
+        @Override
+        public AclFeature getAclFeature() {
+          AclFeature f;
+          if (useDefault) {
+            f = inode.getAclFeature();
+          } else {
+            AclEntry acl = new AclEntry.Builder().setType(AclEntryType.GROUP).
+                setPermission(FsAction.ALL).setName("xxx").build();
+            f = new AclFeature(AclEntryStatusFormat.toInt(
+                Lists.newArrayList(acl)));
+          }
+          return f;
+        }
+
+        @Override
+        public XAttrFeature getXAttrFeature() {
+          return (useDefault) ? inode.getXAttrFeature() : null;
+        }
+
+        @Override
+        public long getModificationTime() {
+          return (useDefault) ? inode.getModificationTime() : 0;
+        }
+
+        @Override
+        public long getAccessTime() {
+          return (useDefault) ? inode.getAccessTime() : 0;
+        }
+      };
+
+    }
+
+    @Override
+    public AccessControlEnforcer getExternalAccessControlEnforcer(
+        AccessControlEnforcer deafultEnforcer) {
+      return new MyAccessControlEnforcer();
+    }
+
+    private boolean useDefault(String[] pathElements) {
+      return (pathElements.length < 2) ||
+          !(pathElements[0].equals("user") && pathElements[1].equals("authz"));
+    }
+
+  }
+
+  @Before
+  public void setUp() throws IOException {
+    CALLED.clear();
+    Configuration conf = new HdfsConfiguration();
+    conf.set(DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY,
+        MyAuthorizationProvider.class.getName());
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
+    EditLogFileOutputStream.setShouldSkipFsyncForTesting(true);
+    miniDFS = new MiniDFSCluster.Builder(conf).build();
+  }
+
+  @After
+  public void cleanUp() throws IOException {
+    CALLED.clear();
+    if (miniDFS != null) {
+      miniDFS.shutdown();
+    }
+    Assert.assertTrue(CALLED.contains("stop"));
+  }
+
+  @Test
+  public void testDelegationToProvider() throws Exception {
+    Assert.assertTrue(CALLED.contains("start"));
+    FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+    fs.mkdirs(new Path("/tmp"));
+    fs.setPermission(new Path("/tmp"), new FsPermission((short) 0777));
+    UserGroupInformation ugi = UserGroupInformation.createUserForTesting("u1",
+        new String[]{"g1"});
+    ugi.doAs(new PrivilegedExceptionAction<Void>() {
+      @Override
+      public Void run() throws Exception {
+        FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+        CALLED.clear();
+        fs.mkdirs(new Path("/tmp/foo"));
+        Assert.assertTrue(CALLED.contains("getAttributes"));
+        Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
+        Assert.assertTrue(CALLED.contains("checkPermission|WRITE|null|null"));
+        CALLED.clear();
+        fs.listStatus(new Path("/tmp/foo"));
+        Assert.assertTrue(CALLED.contains("getAttributes"));
+        Assert.assertTrue(
+            CALLED.contains("checkPermission|null|null|READ_EXECUTE"));
+        CALLED.clear();
+        fs.getAclStatus(new Path("/tmp/foo"));
+        Assert.assertTrue(CALLED.contains("getAttributes"));
+        Assert.assertTrue(CALLED.contains("checkPermission|null|null|null"));
+        return null;
+      }
+    });
+  }
+
+  @Test
+  public void testCustomProvider() throws Exception {
+    FileSystem fs = FileSystem.get(miniDFS.getConfiguration(0));
+    fs.mkdirs(new Path("/user/xxx"));
+    FileStatus status = fs.getFileStatus(new Path("/user/xxx"));
+    Assert.assertEquals(System.getProperty("user.name"), status.getOwner());
+    Assert.assertEquals("supergroup", status.getGroup());
+    Assert.assertEquals(new FsPermission((short)0755), status.getPermission());
+    fs.mkdirs(new Path("/user/authz"));
+    status = fs.getFileStatus(new Path("/user/authz"));
+    Assert.assertEquals("foo", status.getOwner());
+    Assert.assertEquals("bar", status.getGroup());
+    Assert.assertEquals(new FsPermission((short) 0770), status.getPermission());
+  }
+
+}


[33/51] [abbrv] hadoop git commit: Fix CHANGES.txt for HADOOP-11602.

Posted by ka...@apache.org.
Fix CHANGES.txt for HADOOP-11602.


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

Branch: refs/heads/YARN-2139
Commit: 3ca5bd163292e661473017e70b9ca77f5a5b78c0
Parents: 6e891a9
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Mar 24 21:06:26 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Mar 24 21:06:26 2015 +0900

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ca5bd16/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 4f0cf97..cdb88d2 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -415,8 +415,6 @@ Trunk (Unreleased)
     HADOOP-10774. Update KerberosTestUtils for hadoop-auth tests when using
     IBM Java (sangamesh via aw)
 
-    HADOOP-11602. Fix toUpperCase/toLowerCase to use Locale.ENGLISH. (ozawa)
-
     HADOOP-11653. shellprofiles should require .sh extension
     (Brahma Reddy Battula via aw)
 
@@ -1105,6 +1103,8 @@ Release 2.7.0 - UNRELEASED
 
     HADOOP-11670. Regression: s3a auth setup broken. (Adam Budde via stevel)
 
+    HADOOP-11602. Fix toUpperCase/toLowerCase to use Locale.ENGLISH. (ozawa)
+
     HADOOP-11686. MiniKDC cannot change ORG_NAME or ORG_DOMAIN.
     (Duo Zhang via wheat9)
 


[23/51] [abbrv] hadoop git commit: YARN-3241. FairScheduler handles invalid queue names inconsistently. (Zhihai Xu via kasha)

Posted by ka...@apache.org.
YARN-3241. FairScheduler handles invalid queue names inconsistently. (Zhihai Xu via kasha)


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

Branch: refs/heads/YARN-2139
Commit: 2bc097cd14692e6ceb06bff959f28531534eb307
Parents: 6ca1f12
Author: Karthik Kambatla <ka...@apache.org>
Authored: Mon Mar 23 13:22:03 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Mon Mar 23 13:22:03 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +
 .../fair/AllocationFileLoaderService.java       |  8 +-
 .../scheduler/fair/FairScheduler.java           |  2 +
 .../fair/InvalidQueueNameException.java         | 39 ++++++++++
 .../scheduler/fair/QueueManager.java            | 16 ++++
 .../fair/TestAllocationFileLoaderService.java   | 25 ++++++-
 .../scheduler/fair/TestFairScheduler.java       | 78 ++++++++++++++++++++
 .../scheduler/fair/TestQueueManager.java        | 13 +++-
 8 files changed, 181 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bc097cd/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b90109c..b716064 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -94,6 +94,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3269. Yarn.nodemanager.remote-app-log-dir could not be configured to 
     fully qualified path. (Xuan Gong via junping_du)
 
+    YARN-3241. FairScheduler handles "invalid" queue names inconsistently. 
+    (Zhihai Xu via kasha)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bc097cd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.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/fair/AllocationFileLoaderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
index 76fa588..dab6d9f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/AllocationFileLoaderService.java
@@ -426,13 +426,19 @@ public class AllocationFileLoaderService extends AbstractService {
       Map<FSQueueType, Set<String>> configuredQueues,
       Set<String> reservableQueues)
       throws AllocationConfigurationException {
-    String queueName = element.getAttribute("name");
+    String queueName = element.getAttribute("name").trim();
 
     if (queueName.contains(".")) {
       throw new AllocationConfigurationException("Bad fair scheduler config "
           + "file: queue name (" + queueName + ") shouldn't contain period.");
     }
 
+    if (queueName.isEmpty()) {
+      throw new AllocationConfigurationException("Bad fair scheduler config "
+          + "file: queue name shouldn't be empty or "
+          + "consist only of whitespace.");
+    }
+
     if (parentName != null) {
       queueName = parentName + "." + queueName;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bc097cd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.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/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index 1d97983..98a8de2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -701,6 +701,8 @@ public class FairScheduler extends
           appRejectMsg = queueName + " is not a leaf queue";
         }
       }
+    } catch (InvalidQueueNameException qne) {
+      appRejectMsg = qne.getMessage();
     } catch (IOException ioe) {
       appRejectMsg = "Error assigning app to queue " + queueName;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bc097cd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/InvalidQueueNameException.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/fair/InvalidQueueNameException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/InvalidQueueNameException.java
new file mode 100644
index 0000000..fc5ba16
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/InvalidQueueNameException.java
@@ -0,0 +1,39 @@
+/**
+ * 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.yarn.server.resourcemanager.scheduler.fair;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+
+/**
+ * Thrown when Queue Name is malformed.
+ */
+@Private
+@Unstable
+public class InvalidQueueNameException extends IllegalArgumentException {
+  private static final long serialVersionUID = -7306320927804540011L;
+
+  public InvalidQueueNameException(String message) {
+    super(message);
+  }
+
+  public InvalidQueueNameException(String message, Throwable t) {
+    super(message, t);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bc097cd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.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/fair/QueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
index 27e571e..64442ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/QueueManager.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.xml.sax.SAXException;
 
+import com.google.common.annotations.VisibleForTesting;
 /**
  * Maintains a list of queues as well as scheduling parameters for each queue,
  * such as guaranteed share allocations, from the fair scheduler config file.
@@ -155,7 +156,13 @@ public class QueueManager {
 
     // Move up the queue tree until we reach one that exists.
     while (sepIndex != -1) {
+      int prevSepIndex = sepIndex;
       sepIndex = name.lastIndexOf('.', sepIndex-1);
+      String node = name.substring(sepIndex+1, prevSepIndex);
+      if (!isQueueNameValid(node)) {
+        throw new InvalidQueueNameException("Illegal node name at offset " +
+            (sepIndex+1) + " for queue name " + name);
+      }
       FSQueue queue;
       String curName = null;
       curName = name.substring(0, sepIndex);
@@ -401,4 +408,13 @@ public class QueueManager {
     // recursively
     rootQueue.updatePreemptionVariables();
   }
+
+  /**
+   * Check whether queue name is valid,
+   * return true if it is valid, otherwise return false.
+   */
+  @VisibleForTesting
+  boolean isQueueNameValid(String node) {
+    return !node.isEmpty() && node.equals(node.trim());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bc097cd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.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/scheduler/fair/TestAllocationFileLoaderService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java
index 3c166a5..b09573c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestAllocationFileLoaderService.java
@@ -550,7 +550,30 @@ public class TestAllocationFileLoaderService {
     allocLoader.setReloadListener(confHolder);
     allocLoader.reloadAllocations();
   }
-  
+
+  /**
+   * Verify that you can't have the queue name with whitespace only in the
+   * allocations file.
+   */
+  @Test (expected = AllocationConfigurationException.class)
+  public void testQueueNameContainingOnlyWhitespace() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"      \">");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+
+    AllocationFileLoaderService allocLoader = new AllocationFileLoaderService();
+    allocLoader.init(conf);
+    ReloadListener confHolder = new ReloadListener();
+    allocLoader.setReloadListener(confHolder);
+    allocLoader.reloadAllocations();
+  }
 
   @Test
   public void testReservableQueue() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bc097cd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.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/scheduler/fair/TestFairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
index 267fbc2..7600a35 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
@@ -4444,4 +4444,82 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     assertEquals("Incorrect number of perf metrics", 1,
         collector.getRecords().size());
   }
+
+  @Test
+  public void testQueueNameWithTrailingSpace() throws Exception {
+    scheduler.init(conf);
+    scheduler.start();
+    scheduler.reinitialize(conf, resourceManager.getRMContext());
+
+    // only default queue
+    assertEquals(1, scheduler.getQueueManager().getLeafQueues().size());
+
+    // submit app with queue name "A"
+    ApplicationAttemptId appAttemptId1 = createAppAttemptId(1, 1);
+    AppAddedSchedulerEvent appAddedEvent1 = new AppAddedSchedulerEvent(
+        appAttemptId1.getApplicationId(), "A", "user1");
+    scheduler.handle(appAddedEvent1);
+    // submission accepted
+    assertEquals(2, scheduler.getQueueManager().getLeafQueues().size());
+    assertNotNull(scheduler.getSchedulerApplications().get(appAttemptId1.
+        getApplicationId()));
+
+    AppAttemptAddedSchedulerEvent attempAddedEvent =
+        new AppAttemptAddedSchedulerEvent(appAttemptId1, false);
+    scheduler.handle(attempAddedEvent);
+    // That queue should have one app
+    assertEquals(1, scheduler.getQueueManager().getLeafQueue("A", true)
+        .getNumRunnableApps());
+    assertNotNull(scheduler.getSchedulerApp(appAttemptId1));
+
+    // submit app with queue name "A "
+    ApplicationAttemptId appAttemptId2 = createAppAttemptId(2, 1);
+    AppAddedSchedulerEvent appAddedEvent2 = new AppAddedSchedulerEvent(
+        appAttemptId2.getApplicationId(), "A ", "user1");
+    scheduler.handle(appAddedEvent2);
+    // submission rejected
+    assertEquals(2, scheduler.getQueueManager().getLeafQueues().size());
+    assertNull(scheduler.getSchedulerApplications().get(appAttemptId2.
+        getApplicationId()));
+    assertNull(scheduler.getSchedulerApp(appAttemptId2));
+
+    // submit app with queue name "B.C"
+    ApplicationAttemptId appAttemptId3 = createAppAttemptId(3, 1);
+    AppAddedSchedulerEvent appAddedEvent3 = new AppAddedSchedulerEvent(
+        appAttemptId3.getApplicationId(), "B.C", "user1");
+    scheduler.handle(appAddedEvent3);
+    // submission accepted
+    assertEquals(3, scheduler.getQueueManager().getLeafQueues().size());
+    assertNotNull(scheduler.getSchedulerApplications().get(appAttemptId3.
+        getApplicationId()));
+
+    attempAddedEvent =
+        new AppAttemptAddedSchedulerEvent(appAttemptId3, false);
+    scheduler.handle(attempAddedEvent);
+    // That queue should have one app
+    assertEquals(1, scheduler.getQueueManager().getLeafQueue("B.C", true)
+        .getNumRunnableApps());
+    assertNotNull(scheduler.getSchedulerApp(appAttemptId3));
+  }
+
+  @Test
+  public void testEmptyQueueNameInConfigFile() throws IOException {
+    conf.set(FairSchedulerConfiguration.ALLOCATION_FILE, ALLOC_FILE);
+    // set empty queue name
+    PrintWriter out = new PrintWriter(new FileWriter(ALLOC_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<allocations>");
+    out.println("<queue name=\"\">");
+    out.println("</queue>");
+    out.println("</allocations>");
+    out.close();
+    try {
+      scheduler.init(conf);
+      Assert.fail("scheduler init should fail because" +
+          " empty queue name.");
+    } catch (Exception e) {
+      Assert.assertTrue(e.getMessage().contains(
+          "Failed to initialize FairScheduler"));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2bc097cd/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.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/scheduler/fair/TestQueueManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.java
index ef0ec7e..b3ed542 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestQueueManager.java
@@ -123,7 +123,18 @@ public class TestQueueManager {
     assertTrue(queueManager.getParentQueue("root.queue1", false)
         .getChildQueues().isEmpty());
   }
-  
+
+  @Test
+  public void testCheckQueueNodeName() {
+    assertFalse(queueManager.isQueueNameValid(""));
+    assertFalse(queueManager.isQueueNameValid("  "));
+    assertFalse(queueManager.isQueueNameValid(" a"));
+    assertFalse(queueManager.isQueueNameValid("a "));
+    assertFalse(queueManager.isQueueNameValid(" a "));
+    assertTrue(queueManager.isQueueNameValid("a b"));
+    assertTrue(queueManager.isQueueNameValid("a"));
+  }
+
   private void updateConfiguredLeafQueues(QueueManager queueMgr, String... confLeafQueues) {
     AllocationConfiguration allocConf = new AllocationConfiguration(conf);
     allocConf.configuredQueues.get(FSQueueType.LEAF).addAll(Sets.newHashSet(confLeafQueues));


[32/51] [abbrv] hadoop git commit: HADOOP-11609. Correct credential commands info in CommandsManual.html#credential. Contributed by Varun Saxena.

Posted by ka...@apache.org.
HADOOP-11609. Correct credential commands info in CommandsManual.html#credential. 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/6e891a92
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6e891a92
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6e891a92

Branch: refs/heads/YARN-2139
Commit: 6e891a921e00b122390a976dfd13838472a7fcc6
Parents: c6c396f
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Mar 24 20:57:39 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Mar 24 20:57:39 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt                  | 3 +++
 .../java/org/apache/hadoop/security/alias/CredentialShell.java   | 2 +-
 .../hadoop-common/src/site/markdown/CommandsManual.md            | 4 ++--
 3 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e891a92/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 430015d..4f0cf97 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1136,6 +1136,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11729. Fix link to cgroups doc in site.xml. (Masatake Iwasaki via
     ozawa)
 
+    HADOOP-11609. Correct credential commands info in
+    CommandsManual.html#credential. (Varun Saxena via ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e891a92/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
index e8a721f..265ed16 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java
@@ -81,7 +81,7 @@ public class CredentialShell extends Configured implements Tool {
    * <pre>
    * % hadoop credential create alias [-provider providerPath]
    * % hadoop credential list [-provider providerPath]
-   * % hadoop credential delete alias [-provider providerPath] [-i]
+   * % hadoop credential delete alias [-provider providerPath] [-f]
    * </pre>
    * @param args
    * @return 0 if the argument(s) were recognized, 1 otherwise

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e891a92/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md b/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
index 33986ae..207160e 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/CommandsManual.md
@@ -128,8 +128,8 @@ Usage: `hadoop credential <subcommand> [options]`
 
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
-| create *alias* [-v *value*][-provider *provider-path*] | Prompts the user for a credential to be stored as the given alias when a value is not provided via `-v`. The *hadoop.security.credential.provider.path* within the core-site.xml file will be used unless a `-provider` is indicated. |
-| delete *alias* [-i][-provider *provider-path*] | Deletes the credential with the provided alias and optionally warns the user when `--interactive` is used. The *hadoop.security.credential.provider.path* within the core-site.xml file will be used unless a `-provider` is indicated. |
+| create *alias* [-provider *provider-path*] | Prompts the user for a credential to be stored as the given alias. The *hadoop.security.credential.provider.path* within the core-site.xml file will be used unless a `-provider` is indicated. |
+| delete *alias* [-provider *provider-path*] [-f] | Deletes the credential with the provided alias. The *hadoop.security.credential.provider.path* within the core-site.xml file will be used unless a `-provider` is indicated. The command asks for confirmation unless `-f` is specified |
 | list [-provider *provider-path*] | Lists all of the credential aliases The *hadoop.security.credential.provider.path* within the core-site.xml file will be used unless a `-provider` is indicated. |
 
 Command to manage credentials, passwords and secrets within credential providers.


[02/51] [abbrv] hadoop git commit: HDFS-6841. Use Time.monotonicNow() wherever applicable instead of Time.now(). Contributed by Vinayakumar B

Posted by ka...@apache.org.
HDFS-6841. Use Time.monotonicNow() wherever applicable instead of Time.now(). Contributed by Vinayakumar B


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

Branch: refs/heads/YARN-2139
Commit: 75ead273bea8a7dad61c4f99c3a16cab2697c498
Parents: d368d36
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Mar 20 13:31:16 2015 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Mar 20 14:02:09 2015 -0500

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 ++
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  6 +--
 .../org/apache/hadoop/hdfs/DFSOutputStream.java | 40 ++++++++++----------
 .../org/apache/hadoop/hdfs/LeaseRenewer.java    | 14 +++----
 .../hadoop/hdfs/protocol/DatanodeInfo.java      | 38 +++++++++++++++----
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  5 ++-
 .../hadoop/hdfs/server/balancer/Balancer.java   |  8 ++--
 .../BlockInfoContiguousUnderConstruction.java   |  3 +-
 .../server/blockmanagement/BlockManager.java    | 13 ++++---
 .../BlockPlacementPolicyDefault.java            |  8 ++--
 .../blockmanagement/DatanodeDescriptor.java     |  5 ++-
 .../server/blockmanagement/DatanodeManager.java | 12 +++---
 .../blockmanagement/DecommissionManager.java    |  4 +-
 .../blockmanagement/HeartbeatManager.java       |  2 +-
 .../PendingReplicationBlocks.java               |  8 ++--
 .../hdfs/server/datanode/BPServiceActor.java    | 35 +++++++++--------
 .../hdfs/server/datanode/DataXceiver.java       |  6 +--
 .../hdfs/server/namenode/Checkpointer.java      | 10 ++---
 .../server/namenode/EditLogOutputStream.java    |  6 +--
 .../hadoop/hdfs/server/namenode/FSEditLog.java  | 14 +++----
 .../hdfs/server/namenode/FSEditLogLoader.java   | 10 ++---
 .../hdfs/server/namenode/FSImageFormat.java     | 16 ++++----
 .../hdfs/server/namenode/FSNamesystem.java      | 24 +++++++-----
 .../hdfs/server/namenode/LeaseManager.java      |  8 ++--
 .../hdfs/server/namenode/NamenodeFsck.java      |  6 +--
 .../hdfs/server/namenode/ha/EditLogTailer.java  | 16 ++++----
 .../org/apache/hadoop/hdfs/web/JsonUtil.java    |  2 +
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |  1 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     | 27 +++++++++++--
 .../org/apache/hadoop/hdfs/MiniDFSCluster.java  |  2 +-
 .../org/apache/hadoop/hdfs/TestGetBlocks.java   | 14 +++----
 .../hdfs/TestInjectionForSimulatedStorage.java  |  4 +-
 .../java/org/apache/hadoop/hdfs/TestLease.java  |  4 +-
 .../apache/hadoop/hdfs/TestLeaseRenewer.java    | 10 ++---
 .../hadoop/hdfs/TestParallelReadUtil.java       |  4 +-
 .../org/apache/hadoop/hdfs/TestReplication.java |  4 +-
 .../hdfs/server/balancer/TestBalancer.java      |  8 ++--
 .../blockmanagement/BlockManagerTestUtil.java   |  2 +-
 .../TestBlockInfoUnderConstruction.java         | 31 +++++++--------
 .../blockmanagement/TestHeartbeatHandling.java  | 20 +++++-----
 .../blockmanagement/TestHostFileManager.java    |  3 +-
 .../server/blockmanagement/TestNodeCount.java   |  4 +-
 .../TestOverReplicatedBlocks.java               | 11 +++---
 .../blockmanagement/TestReplicationPolicy.java  | 34 +++++++++--------
 .../server/datanode/BlockReportTestBase.java    |  8 ++--
 .../server/datanode/TestBlockReplacement.java   |  8 ++--
 .../namenode/TestNamenodeCapacityReport.java    |  5 ++-
 .../namenode/metrics/TestNameNodeMetrics.java   | 15 +++++---
 48 files changed, 304 insertions(+), 237 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 0ab14f2..e82c4c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1229,6 +1229,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7957. Truncate should verify quota before making changes. (jing9)
 
+    HDFS-6841. Use Time.monotonicNow() wherever applicable instead of Time.now()
+    (Vinayakumar B via kihwal)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 3236771..70f66bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -885,7 +885,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       if (filesBeingWritten.isEmpty()) {
         return;
       }
-      lastLeaseRenewal = Time.now();
+      lastLeaseRenewal = Time.monotonicNow();
     }
   }
 
@@ -902,7 +902,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
         return true;
       } catch (IOException e) {
         // Abort if the lease has already expired. 
-        final long elapsed = Time.now() - getLastLeaseRenewal();
+        final long elapsed = Time.monotonicNow() - getLastLeaseRenewal();
         if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
           LOG.warn("Failed to renew lease for " + clientName + " for "
               + (elapsed/1000) + " seconds (>= hard-limit ="
@@ -1020,7 +1020,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#getServerDefaults()
    */
   public FsServerDefaults getServerDefaults() throws IOException {
-    long now = Time.now();
+    long now = Time.monotonicNow();
     if (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD) {
       serverDefaults = namenode.getServerDefaults();
       serverDefaultsLastUpdate = now;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 c3df897..d7d59af 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
@@ -382,7 +382,7 @@ public class DFSOutputStream extends FSOutputSummer
      */
     @Override
     public void run() {
-      long lastPacket = Time.now();
+      long lastPacket = Time.monotonicNow();
       TraceScope scope = NullScope.INSTANCE;
       while (!streamerClosed && dfsClient.clientRunning) {
         // if the Responder encountered an error, shutdown Responder
@@ -406,7 +406,7 @@ public class DFSOutputStream extends FSOutputSummer
 
           synchronized (dataQueue) {
             // wait for a packet to be sent.
-            long now = Time.now();
+            long now = Time.monotonicNow();
             while ((!streamerClosed && !hasError && dfsClient.clientRunning 
                 && dataQueue.size() == 0 && 
                 (stage != BlockConstructionStage.DATA_STREAMING || 
@@ -422,7 +422,7 @@ public class DFSOutputStream extends FSOutputSummer
                 DFSClient.LOG.warn("Caught exception ", e);
               }
               doSleep = false;
-              now = Time.now();
+              now = Time.monotonicNow();
             }
             if (streamerClosed || hasError || !dfsClient.clientRunning) {
               continue;
@@ -521,7 +521,7 @@ public class DFSOutputStream extends FSOutputSummer
           } finally {
             writeScope.close();
           }
-          lastPacket = Time.now();
+          lastPacket = Time.monotonicNow();
           
           // update bytesSent
           long tmpBytesSent = one.getLastByteOffsetBlock();
@@ -760,8 +760,8 @@ public class DFSOutputStream extends FSOutputSummer
               // the local node or the only one in the pipeline.
               if (PipelineAck.isRestartOOBStatus(reply) &&
                   shouldWaitForRestart(i)) {
-                restartDeadline = dfsClient.getConf().datanodeRestartTimeout +
-                    Time.now();
+                restartDeadline = dfsClient.getConf().datanodeRestartTimeout
+                    + Time.monotonicNow();
                 setRestartingNodeIndex(i);
                 String message = "A datanode is restarting: " + targets[i];
                 DFSClient.LOG.info(message);
@@ -1175,7 +1175,7 @@ public class DFSOutputStream extends FSOutputSummer
             errorIndex = -1;
           }
           // still within the deadline
-          if (Time.now() < restartDeadline) {
+          if (Time.monotonicNow() < restartDeadline) {
             continue; // with in the deadline
           }
           // expired. declare the restarting node dead
@@ -1226,14 +1226,12 @@ public class DFSOutputStream extends FSOutputSummer
         errorIndex = -1;
         success = false;
 
-        long startTime = Time.now();
         DatanodeInfo[] excluded =
             excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
             .keySet()
             .toArray(new DatanodeInfo[0]);
         block = oldBlock;
-        lb = locateFollowingBlock(startTime,
-            excluded.length > 0 ? excluded : null);
+        lb = locateFollowingBlock(excluded.length > 0 ? excluded : null);
         block = lb.getBlock();
         block.setNumBytes(0);
         bytesSent = 0;
@@ -1380,7 +1378,7 @@ public class DFSOutputStream extends FSOutputSummer
           // Check whether there is a restart worth waiting for.
           if (checkRestart && shouldWaitForRestart(errorIndex)) {
             restartDeadline = dfsClient.getConf().datanodeRestartTimeout +
-                Time.now();
+                Time.monotonicNow();
             restartingNodeIndex.set(errorIndex);
             errorIndex = -1;
             DFSClient.LOG.info("Waiting for the datanode to be restarted: " +
@@ -1430,13 +1428,12 @@ public class DFSOutputStream extends FSOutputSummer
       }
     }
 
-    private LocatedBlock locateFollowingBlock(long start,
-        DatanodeInfo[] excludedNodes)  throws IOException {
+    private LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)  throws IOException {
       int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
       long sleeptime = dfsClient.getConf().
           blockWriteLocateFollowingInitialDelayMs;
       while (true) {
-        long localstart = Time.now();
+        long localstart = Time.monotonicNow();
         while (true) {
           try {
             return dfsClient.namenode.addBlock(src, dfsClient.clientName,
@@ -1460,10 +1457,10 @@ public class DFSOutputStream extends FSOutputSummer
               } else {
                 --retries;
                 DFSClient.LOG.info("Exception while adding a block", e);
-                if (Time.now() - localstart > 5000) {
+                long elapsed = Time.monotonicNow() - localstart;
+                if (elapsed > 5000) {
                   DFSClient.LOG.info("Waiting for replication for "
-                      + (Time.now() - localstart) / 1000
-                      + " seconds");
+                      + (elapsed / 1000) + " seconds");
                 }
                 try {
                   DFSClient.LOG.warn("NotReplicatedYetException sleeping " + src
@@ -2253,7 +2250,7 @@ public class DFSOutputStream extends FSOutputSummer
   // should be called holding (this) lock since setTestFilename() may 
   // be called during unit tests
   private void completeFile(ExtendedBlock last) throws IOException {
-    long localstart = Time.now();
+    long localstart = Time.monotonicNow();
     long sleeptime = dfsClient.getConf().
         blockWriteLocateFollowingInitialDelayMs;
     boolean fileComplete = false;
@@ -2263,8 +2260,9 @@ public class DFSOutputStream extends FSOutputSummer
           dfsClient.namenode.complete(src, dfsClient.clientName, last, fileId);
       if (!fileComplete) {
         final int hdfsTimeout = dfsClient.getHdfsTimeout();
-        if (!dfsClient.clientRunning ||
-              (hdfsTimeout > 0 && localstart + hdfsTimeout < Time.now())) {
+        if (!dfsClient.clientRunning
+            || (hdfsTimeout > 0 
+                && localstart + hdfsTimeout < Time.monotonicNow())) {
             String msg = "Unable to close file because dfsclient " +
                           " was unable to contact the HDFS servers." +
                           " clientRunning " + dfsClient.clientRunning +
@@ -2280,7 +2278,7 @@ public class DFSOutputStream extends FSOutputSummer
           retries--;
           Thread.sleep(sleeptime);
           sleeptime *= 2;
-          if (Time.now() - localstart > 5000) {
+          if (Time.monotonicNow() - localstart > 5000) {
             DFSClient.LOG.info("Could not complete " + src + " retrying...");
           }
         } catch (InterruptedException ie) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
index f8f337c..e767501 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/LeaseRenewer.java
@@ -278,7 +278,7 @@ class LeaseRenewer {
   /** Is the empty period longer than the grace period? */  
   private synchronized boolean isRenewerExpired() {
     return emptyTime != Long.MAX_VALUE
-        && Time.now() - emptyTime > gracePeriod;
+        && Time.monotonicNow() - emptyTime > gracePeriod;
   }
 
   synchronized void put(final long inodeId, final DFSOutputStream out,
@@ -346,7 +346,7 @@ class LeaseRenewer {
           }
         }
         //discover the first time that all file-being-written maps are empty.
-        emptyTime = Time.now();
+        emptyTime = Time.monotonicNow();
       }
     }
   }
@@ -361,7 +361,7 @@ class LeaseRenewer {
       }
       if (emptyTime == Long.MAX_VALUE) {
         //discover the first time that the client list is empty.
-        emptyTime = Time.now();
+        emptyTime = Time.monotonicNow();
       }
     }
 
@@ -434,9 +434,9 @@ class LeaseRenewer {
    * when the lease period is half over.
    */
   private void run(final int id) throws InterruptedException {
-    for(long lastRenewed = Time.now(); !Thread.interrupted();
+    for(long lastRenewed = Time.monotonicNow(); !Thread.interrupted();
         Thread.sleep(getSleepPeriod())) {
-      final long elapsed = Time.now() - lastRenewed;
+      final long elapsed = Time.monotonicNow() - lastRenewed;
       if (elapsed >= getRenewalTime()) {
         try {
           renew();
@@ -444,7 +444,7 @@ class LeaseRenewer {
             LOG.debug("Lease renewer daemon for " + clientsString()
                 + " with renew id " + id + " executed");
           }
-          lastRenewed = Time.now();
+          lastRenewed = Time.monotonicNow();
         } catch (SocketTimeoutException ie) {
           LOG.warn("Failed to renew lease for " + clientsString() + " for "
               + (elapsed/1000) + " seconds.  Aborting ...", ie);
@@ -479,7 +479,7 @@ class LeaseRenewer {
         // registered with this renewer, stop the daemon after the grace
         // period.
         if (!clientsRunning() && emptyTime == Long.MAX_VALUE) {
-          emptyTime = Time.now();
+          emptyTime = Time.monotonicNow();
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
index 9fcada7..5ded26b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeInfo.java
@@ -49,6 +49,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
   private long cacheCapacity;
   private long cacheUsed;
   private long lastUpdate;
+  private long lastUpdateMonotonic;
   private int xceiverCount;
   private String location = NetworkTopology.DEFAULT_RACK;
   private String softwareVersion;
@@ -91,6 +92,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.cacheCapacity = from.getCacheCapacity();
     this.cacheUsed = from.getCacheUsed();
     this.lastUpdate = from.getLastUpdate();
+    this.lastUpdateMonotonic = from.getLastUpdateMonotonic();
     this.xceiverCount = from.getXceiverCount();
     this.location = from.getNetworkLocation();
     this.adminState = from.getAdminState();
@@ -105,6 +107,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.cacheCapacity = 0L;
     this.cacheUsed = 0L;
     this.lastUpdate = 0L;
+    this.lastUpdateMonotonic = 0L;
     this.xceiverCount = 0;
     this.adminState = null;    
   }
@@ -117,13 +120,13 @@ public class DatanodeInfo extends DatanodeID implements Node {
   public DatanodeInfo(DatanodeID nodeID, String location,
       final long capacity, final long dfsUsed, final long remaining,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
-      final long lastUpdate, final int xceiverCount,
-      final AdminStates adminState) {
+      final long lastUpdate, final long lastUpdateMonotonic,
+      final int xceiverCount, final AdminStates adminState) {
     this(nodeID.getIpAddr(), nodeID.getHostName(), nodeID.getDatanodeUuid(),
         nodeID.getXferPort(), nodeID.getInfoPort(), nodeID.getInfoSecurePort(),
         nodeID.getIpcPort(), capacity, dfsUsed, remaining, blockPoolUsed,
-        cacheCapacity, cacheUsed, lastUpdate, xceiverCount, location,
-        adminState);
+        cacheCapacity, cacheUsed, lastUpdate, lastUpdateMonotonic,
+        xceiverCount, location, adminState);
   }
 
   /** Constructor */
@@ -132,8 +135,9 @@ public class DatanodeInfo extends DatanodeID implements Node {
       final int infoSecurePort, final int ipcPort,
       final long capacity, final long dfsUsed, final long remaining,
       final long blockPoolUsed, final long cacheCapacity, final long cacheUsed,
-      final long lastUpdate, final int xceiverCount,
-      final String networkLocation, final AdminStates adminState) {
+      final long lastUpdate, final long lastUpdateMonotonic,
+      final int xceiverCount, final String networkLocation,
+      final AdminStates adminState) {
     super(ipAddr, hostName, datanodeUuid, xferPort, infoPort,
             infoSecurePort, ipcPort);
     this.capacity = capacity;
@@ -143,6 +147,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
     this.cacheCapacity = cacheCapacity;
     this.cacheUsed = cacheUsed;
     this.lastUpdate = lastUpdate;
+    this.lastUpdateMonotonic = lastUpdateMonotonic;
     this.xceiverCount = xceiverCount;
     this.location = networkLocation;
     this.adminState = adminState;
@@ -223,9 +228,26 @@ public class DatanodeInfo extends DatanodeID implements Node {
     return DFSUtil.getPercentRemaining(getCacheRemaining(), cacheCapacity);
   }
 
-  /** The time when this information was accurate. */
+  /**
+   * Get the last update timestamp.
+   * Return value is suitable for Date conversion.
+   */
   public long getLastUpdate() { return lastUpdate; }
 
+  /** 
+   * The time when this information was accurate. <br>
+   * Ps: So return value is ideal for calculation of time differences.
+   * Should not be used to convert to Date.  
+   */
+  public long getLastUpdateMonotonic() { return lastUpdateMonotonic;}
+
+  /**
+   * Set lastUpdate monotonic time
+   */
+  public void setLastUpdateMonotonic(long lastUpdateMonotonic) {
+    this.lastUpdateMonotonic = lastUpdateMonotonic;
+  }
+
   /** number of active connections */
   public int getXceiverCount() { return xceiverCount; }
 
@@ -437,7 +459,7 @@ public class DatanodeInfo extends DatanodeID implements Node {
    * @return true if the node is stale
    */
   public boolean isStale(long staleInterval) {
-    return (Time.now() - lastUpdate) >= staleInterval;
+    return (Time.monotonicNow() - lastUpdateMonotonic) >= staleInterval;
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 c428c2b..fad1d2c 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
@@ -642,8 +642,8 @@ public class PBHelper {
         di.hasLocation() ? di.getLocation() : null , 
         di.getCapacity(),  di.getDfsUsed(),  di.getRemaining(),
         di.getBlockPoolUsed(), di.getCacheCapacity(), di.getCacheUsed(),
-        di.getLastUpdate(), di.getXceiverCount(),
-        PBHelper.convert(di.getAdminState())); 
+        di.getLastUpdate(), di.getLastUpdateMonotonic(),
+        di.getXceiverCount(), PBHelper.convert(di.getAdminState()));
   }
   
   static public DatanodeInfoProto convertDatanodeInfo(DatanodeInfo di) {
@@ -704,6 +704,7 @@ public class PBHelper {
         .setCacheCapacity(info.getCacheCapacity())
         .setCacheUsed(info.getCacheUsed())
         .setLastUpdate(info.getLastUpdate())
+        .setLastUpdateMonotonic(info.getLastUpdateMonotonic())
         .setXceiverCount(info.getXceiverCount())
         .setAdminState(PBHelper.convert(info.getAdminState()))
         .build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
index 90212a3..bc7e448 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
@@ -672,7 +672,7 @@ public class Balancer {
      */
     @Override
     public int run(String[] args) {
-      final long startTime = Time.now();
+      final long startTime = Time.monotonicNow();
       final Configuration conf = getConf();
 
       try {
@@ -687,8 +687,10 @@ public class Balancer {
         System.out.println(e + ".  Exiting ...");
         return ExitStatus.INTERRUPTED.getExitCode();
       } finally {
-        System.out.format("%-24s ", DateFormat.getDateTimeInstance().format(new Date()));
-        System.out.println("Balancing took " + time2Str(Time.now()-startTime));
+        System.out.format("%-24s ",
+            DateFormat.getDateTimeInstance().format(new Date()));
+        System.out.println("Balancing took "
+            + time2Str(Time.monotonicNow() - startTime));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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
index ae809a5..92153ab 100644
--- 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
@@ -315,7 +315,8 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
         continue;
       }
       final ReplicaUnderConstruction ruc = replicas.get(i);
-      final long lastUpdate = ruc.getExpectedStorageLocation().getDatanodeDescriptor().getLastUpdate(); 
+      final long lastUpdate = ruc.getExpectedStorageLocation()
+          .getDatanodeDescriptor().getLastUpdateMonotonic();
       if (lastUpdate > mostRecentLastUpdate) {
         primaryNodeIndex = i;
         primary = ruc;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 a2fe165..674c0ea 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
@@ -1772,7 +1772,7 @@ public class BlockManager {
       final DatanodeStorage storage,
       final BlockListAsLongs newReport) throws IOException {
     namesystem.writeLock();
-    final long startTime = Time.now(); //after acquiring write lock
+    final long startTime = Time.monotonicNow(); //after acquiring write lock
     final long endTime;
     DatanodeDescriptor node;
     Collection<Block> invalidatedBlocks = null;
@@ -1810,7 +1810,7 @@ public class BlockManager {
       
       storageInfo.receivedBlockReport();
     } finally {
-      endTime = Time.now();
+      endTime = Time.monotonicNow();
       namesystem.writeUnlock();
     }
 
@@ -1840,7 +1840,7 @@ public class BlockManager {
     if (getPostponedMisreplicatedBlocksCount() == 0) {
       return;
     }
-    long startTimeRescanPostponedMisReplicatedBlocks = Time.now();
+    long startTimeRescanPostponedMisReplicatedBlocks = Time.monotonicNow();
     long startPostponedMisReplicatedBlocksCount =
         getPostponedMisreplicatedBlocksCount();
     namesystem.writeLock();
@@ -1900,7 +1900,7 @@ public class BlockManager {
       long endPostponedMisReplicatedBlocksCount =
           getPostponedMisreplicatedBlocksCount();
       LOG.info("Rescan of postponedMisreplicatedBlocks completed in " +
-          (Time.now() - startTimeRescanPostponedMisReplicatedBlocks) +
+          (Time.monotonicNow() - startTimeRescanPostponedMisReplicatedBlocks) +
           " msecs. " + endPostponedMisReplicatedBlocksCount +
           " blocks are left. " + (startPostponedMisReplicatedBlocksCount -
           endPostponedMisReplicatedBlocksCount) + " blocks are removed.");
@@ -2648,7 +2648,7 @@ public class BlockManager {
   private void processMisReplicatesAsync() throws InterruptedException {
     long nrInvalid = 0, nrOverReplicated = 0;
     long nrUnderReplicated = 0, nrPostponed = 0, nrUnderConstruction = 0;
-    long startTimeMisReplicatedScan = Time.now();
+    long startTimeMisReplicatedScan = Time.monotonicNow();
     Iterator<BlockInfoContiguous> blocksItr = blocksMap.getBlocks().iterator();
     long totalBlocks = blocksMap.size();
     replicationQueuesInitProgress = 0;
@@ -2706,7 +2706,8 @@ public class BlockManager {
           NameNode.stateChangeLog
               .info("STATE* Replication Queue initialization "
                   + "scan for invalid, over- and under-replicated blocks "
-                  + "completed in " + (Time.now() - startTimeMisReplicatedScan)
+                  + "completed in "
+                  + (Time.monotonicNow() - startTimeMisReplicatedScan)
                   + " msec");
           break;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
index cb17596..3262772 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicyDefault.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.util.*;
 
@@ -884,7 +884,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
       Collection<DatanodeStorageInfo> second,
       final List<StorageType> excessTypes) {
     long oldestHeartbeat =
-      now() - heartbeatInterval * tolerateHeartbeatMultiplier;
+      monotonicNow() - heartbeatInterval * tolerateHeartbeatMultiplier;
     DatanodeStorageInfo oldestHeartbeatStorage = null;
     long minSpace = Long.MAX_VALUE;
     DatanodeStorageInfo minSpaceStorage = null;
@@ -898,8 +898,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
 
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       long free = node.getRemaining();
-      long lastHeartbeat = node.getLastUpdate();
-      if(lastHeartbeat < oldestHeartbeat) {
+      long lastHeartbeat = node.getLastUpdateMonotonic();
+      if (lastHeartbeat < oldestHeartbeat) {
         oldestHeartbeat = lastHeartbeat;
         oldestHeartbeatStorage = storage;
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 c0a17b1..3f143e7 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
@@ -405,7 +405,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
     setCacheCapacity(cacheCapacity);
     setCacheUsed(cacheUsed);
     setXceiverCount(xceiverCount);
-    setLastUpdate(Time.now());    
+    setLastUpdate(Time.now());
+    setLastUpdateMonotonic(Time.monotonicNow());
     this.volumeFailures = volFailures;
     this.volumeFailureSummary = volumeFailureSummary;
     for (StorageReport report : reports) {
@@ -420,7 +421,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
       totalBlockPoolUsed += report.getBlockPoolUsed();
       totalDfsUsed += report.getDfsUsed();
     }
-    rollBlocksScheduled(getLastUpdate());
+    rollBlocksScheduled(getLastUpdateMonotonic());
 
     // Update total metrics for the node.
     setCapacity(totalCapacity);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 9179ff0..f68c4fd 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
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import static org.apache.hadoop.util.Time.monotonicNow;
+
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.net.InetAddresses;
@@ -43,7 +45,6 @@ import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.*;
 import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
 import org.apache.hadoop.util.ReflectionUtils;
-import org.apache.hadoop.util.Time;
 
 import java.io.IOException;
 import java.io.PrintWriter;
@@ -581,8 +582,8 @@ public class DatanodeManager {
 
   /** Is the datanode dead? */
   boolean isDatanodeDead(DatanodeDescriptor node) {
-    return (node.getLastUpdate() <
-            (Time.now() - heartbeatExpireInterval));
+    return (node.getLastUpdateMonotonic() <
+            (monotonicNow() - heartbeatExpireInterval));
   }
 
   /** Add a datanode. */
@@ -1299,7 +1300,7 @@ public class DatanodeManager {
                 .getAddress().getHostAddress(), addr.getHostName(), "",
                 addr.getPort() == 0 ? defaultXferPort : addr.getPort(),
                 defaultInfoPort, defaultInfoSecurePort, defaultIpcPort));
-        dn.setLastUpdate(0); // Consider this node dead for reporting
+        setDatanodeDead(dn);
         nodes.add(dn);
       }
     }
@@ -1332,6 +1333,7 @@ public class DatanodeManager {
   
   private void setDatanodeDead(DatanodeDescriptor node) {
     node.setLastUpdate(0);
+    node.setLastUpdateMonotonic(0);
   }
 
   /** Handle heartbeat from datanodes. */
@@ -1437,7 +1439,7 @@ public class DatanodeManager {
               blockPoolId, blks));
         }
         boolean sendingCachingCommands = false;
-        long nowMs = Time.monotonicNow();
+        long nowMs = monotonicNow();
         if (shouldSendCachingCommands && 
             ((nowMs - nodeinfo.getLastCachingDirectiveSentTimeMs()) >=
                 timeBetweenResendingCachingDirectivesMs)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 71c88f1..9355329 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
@@ -45,7 +45,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static com.google.common.base.Preconditions.checkArgument;
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 /**
  * Manages datanode decommissioning. A background monitor thread 
@@ -208,7 +208,7 @@ public class DecommissionManager {
         }
         // Update DN stats maintained by HeartbeatManager
         hbManager.startDecommission(node);
-        node.decommissioningStatus.setStartTime(now());
+        node.decommissioningStatus.setStartTime(monotonicNow());
         pendingNodes.add(node);
       }
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
index d60a39b..d2905a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/HeartbeatManager.java
@@ -353,7 +353,7 @@ class HeartbeatManager implements DatanodeStatistics {
     public void run() {
       while(namesystem.isRunning()) {
         try {
-          final long now = Time.now();
+          final long now = Time.monotonicNow();
           if (lastHeartbeatCheck + heartbeatRecheckInterval < now) {
             heartbeatCheck();
             lastHeartbeatCheck = now;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 57c2962..796b878 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
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.PrintWriter;
 import java.sql.Time;
@@ -177,7 +177,7 @@ class PendingReplicationBlocks {
     private final List<DatanodeDescriptor> targets;
 
     PendingBlockInfo(DatanodeDescriptor[] targets) {
-      this.timeStamp = now();
+      this.timeStamp = monotonicNow();
       this.targets = targets == null ? new ArrayList<DatanodeDescriptor>()
           : new ArrayList<DatanodeDescriptor>(Arrays.asList(targets));
     }
@@ -187,7 +187,7 @@ class PendingReplicationBlocks {
     }
 
     void setTimeStamp() {
-      timeStamp = now();
+      timeStamp = monotonicNow();
     }
 
     void incrementReplicas(DatanodeDescriptor... newTargets) {
@@ -234,7 +234,7 @@ class PendingReplicationBlocks {
       synchronized (pendingReplications) {
         Iterator<Map.Entry<Block, PendingBlockInfo>> iter =
                                     pendingReplications.entrySet().iterator();
-        long now = now();
+        long now = monotonicNow();
         if(LOG.isDebugEnabled()) {
           LOG.debug("PendingReplicationMonitor checking Q");
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 3c20f6b..90f2fe6 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
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.datanode;
 
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -57,7 +57,6 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.hadoop.util.VersionUtil;
 
@@ -249,7 +248,7 @@ class BPServiceActor implements Runnable {
    */
   void scheduleBlockReport(long delay) {
     if (delay > 0) { // send BR after random delay
-      lastBlockReport = Time.now()
+      lastBlockReport = monotonicNow()
       - ( dnConf.blockReportInterval - DFSUtil.getRandom().nextInt((int)(delay)));
     } else { // send at next heartbeat
       lastBlockReport = lastHeartbeat - dnConf.blockReportInterval;
@@ -291,14 +290,14 @@ class BPServiceActor implements Runnable {
 
     // Send incremental block reports to the Namenode outside the lock
     boolean success = false;
-    final long startTime = Time.monotonicNow();
+    final long startTime = monotonicNow();
     try {
       bpNamenode.blockReceivedAndDeleted(bpRegistration,
           bpos.getBlockPoolId(),
           reports.toArray(new StorageReceivedDeletedBlocks[reports.size()]));
       success = true;
     } finally {
-      dn.getMetrics().addIncrementalBlockReport(Time.monotonicNow()-startTime);
+      dn.getMetrics().addIncrementalBlockReport(monotonicNow() - startTime);
       if (!success) {
         synchronized (pendingIncrementalBRperStorage) {
           for (StorageReceivedDeletedBlocks report : reports) {
@@ -442,7 +441,7 @@ class BPServiceActor implements Runnable {
    */
   List<DatanodeCommand> blockReport() throws IOException {
     // send block report if timer has expired.
-    final long startTime = now();
+    final long startTime = monotonicNow();
     if (startTime - lastBlockReport <= dnConf.blockReportInterval) {
       return null;
     }
@@ -456,7 +455,7 @@ class BPServiceActor implements Runnable {
     reportReceivedDeletedBlocks();
     lastDeletedReport = startTime;
 
-    long brCreateStartTime = now();
+    long brCreateStartTime = monotonicNow();
     Map<DatanodeStorage, BlockListAsLongs> perVolumeBlockLists =
         dn.getFSDataset().getBlockReports(bpos.getBlockPoolId());
 
@@ -476,7 +475,7 @@ class BPServiceActor implements Runnable {
     int numReportsSent = 0;
     int numRPCs = 0;
     boolean success = false;
-    long brSendStartTime = now();
+    long brSendStartTime = monotonicNow();
     try {
       if (totalBlockCount < dnConf.blockReportSplitThreshold) {
         // Below split threshold, send all reports in a single message.
@@ -503,7 +502,7 @@ class BPServiceActor implements Runnable {
       success = true;
     } finally {
       // Log the block report processing stats from Datanode perspective
-      long brSendCost = now() - brSendStartTime;
+      long brSendCost = monotonicNow() - brSendStartTime;
       long brCreateCost = brSendStartTime - brCreateStartTime;
       dn.getMetrics().addBlockReport(brSendCost);
       final int nCmds = cmds.size();
@@ -539,7 +538,7 @@ class BPServiceActor implements Runnable {
        *   1) normal like 9:20:18, next report should be at 10:20:14
        *   2) unexpected like 11:35:43, next report should be at 12:20:14
        */
-      lastBlockReport += (now() - lastBlockReport) /
+      lastBlockReport += (monotonicNow() - lastBlockReport) /
           dnConf.blockReportInterval * dnConf.blockReportInterval;
     }
   }
@@ -551,7 +550,7 @@ class BPServiceActor implements Runnable {
     }
     // send cache report if timer has expired.
     DatanodeCommand cmd = null;
-    final long startTime = Time.monotonicNow();
+    final long startTime = monotonicNow();
     if (startTime - lastCacheReport > dnConf.cacheReportInterval) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Sending cacheReport from service actor: " + this);
@@ -560,10 +559,10 @@ class BPServiceActor implements Runnable {
 
       String bpid = bpos.getBlockPoolId();
       List<Long> blockIds = dn.getFSDataset().getCacheReport(bpid);
-      long createTime = Time.monotonicNow();
+      long createTime = monotonicNow();
 
       cmd = bpNamenode.cacheReport(bpRegistration, bpid, blockIds);
-      long sendTime = Time.monotonicNow();
+      long sendTime = monotonicNow();
       long createCost = createTime - startTime;
       long sendCost = sendTime - createTime;
       dn.getMetrics().addCacheReport(sendCost);
@@ -670,7 +669,7 @@ class BPServiceActor implements Runnable {
     //
     while (shouldRun()) {
       try {
-        final long startTime = now();
+        final long startTime = monotonicNow();
 
         //
         // Every so often, send heartbeat or block-report
@@ -687,7 +686,7 @@ class BPServiceActor implements Runnable {
           if (!dn.areHeartbeatsDisabledForTests()) {
             HeartbeatResponse resp = sendHeartBeat();
             assert resp != null;
-            dn.getMetrics().addHeartbeat(now() - startTime);
+            dn.getMetrics().addHeartbeat(monotonicNow() - startTime);
 
             // If the state of this NN has changed (eg STANDBY->ACTIVE)
             // then let the BPOfferService update itself.
@@ -703,10 +702,10 @@ class BPServiceActor implements Runnable {
               handleRollingUpgradeStatus(resp);
             }
 
-            long startProcessCommands = now();
+            long startProcessCommands = monotonicNow();
             if (!processCommand(resp.getCommands()))
               continue;
-            long endProcessCommands = now();
+            long endProcessCommands = monotonicNow();
             if (endProcessCommands - startProcessCommands > 2000) {
               LOG.info("Took " + (endProcessCommands - startProcessCommands)
                   + "ms to process " + resp.getCommands().length
@@ -731,7 +730,7 @@ class BPServiceActor implements Runnable {
         // or work arrives, and then iterate again.
         //
         long waitTime = dnConf.heartBeatInterval - 
-        (Time.now() - lastHeartbeat);
+        (monotonicNow() - lastHeartbeat);
         synchronized(pendingIncrementalBRperStorage) {
           if (waitTime > 0 && !sendImmediateIBR) {
             try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 84504fb..cf1b6be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -25,7 +25,7 @@ import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SU
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
 import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.DO_NOT_USE_RECEIPT_VERIFICATION;
 import static org.apache.hadoop.hdfs.server.datanode.DataNode.DN_CLIENTTRACE_FORMAT;
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
@@ -247,7 +247,7 @@ class DataXceiver extends Receiver implements Runnable {
           peer.setReadTimeout(dnConf.socketTimeout);
         }
 
-        opStartTime = now();
+        opStartTime = monotonicNow();
         processOp(op);
         ++opsProcessed;
       } while ((peer != null) &&
@@ -1202,7 +1202,7 @@ class DataXceiver extends Receiver implements Runnable {
   }
 
   private long elapsed() {
-    return now() - opStartTime;
+    return monotonicNow() - opStartTime;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
index 9327f43..011a459 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_BACKUP_HTTP_ADDRESS_KEY;
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.File;
 import java.io.IOException;
@@ -135,11 +135,11 @@ class Checkpointer extends Daemon {
 
     long lastCheckpointTime = 0;
     if (!backupNode.shouldCheckpointAtStartup()) {
-      lastCheckpointTime = now();
+      lastCheckpointTime = monotonicNow();
     }
     while(shouldRun) {
       try {
-        long now = now();
+        long now = monotonicNow();
         boolean shouldCheckpoint = false;
         if(now >= lastCheckpointTime + periodMSec) {
           shouldCheckpoint = true;
@@ -182,7 +182,7 @@ class Checkpointer extends Daemon {
     BackupImage bnImage = getFSImage();
     NNStorage bnStorage = bnImage.getStorage();
 
-    long startTime = now();
+    long startTime = monotonicNow();
     bnImage.freezeNamespaceAtNextRoll();
     
     NamenodeCommand cmd = 
@@ -276,7 +276,7 @@ class Checkpointer extends Daemon {
     
     long imageSize = bnImage.getStorage().getFsImageName(txid).length();
     LOG.info("Checkpoint completed in "
-        + (now() - startTime)/1000 + " seconds."
+        + (monotonicNow() - startTime)/1000 + " seconds."
         + " New Image Size: " + imageSize);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
index 5e6d9d8..b4ca2d6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogOutputStream.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 import java.io.Closeable;
 
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -109,9 +109,9 @@ public abstract class EditLogOutputStream implements Closeable {
   
   public void flush(boolean durable) throws IOException {
     numSync++;
-    long start = now();
+    long start = monotonicNow();
     flushAndSync(durable);
-    long end = now();
+    long end = monotonicNow();
     totalTimeSync += (end - start);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index 9d487e5..df9c585 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -18,7 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.util.ExitUtil.terminate;
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.IOException;
 import java.lang.reflect.Constructor;
@@ -230,7 +230,7 @@ public class FSEditLog implements LogsPurgeable {
     this.conf = conf;
     this.storage = storage;
     metrics = NameNode.getNameNodeMetrics();
-    lastPrintTime = now();
+    lastPrintTime = monotonicNow();
      
     // If this list is empty, an error will be thrown on first use
     // of the editlog, as no journals will exist
@@ -486,14 +486,14 @@ public class FSEditLog implements LogsPurgeable {
     //
     TransactionId id = myTransactionId.get();
     id.txid = txid;
-    return now();
+    return monotonicNow();
   }
   
   private void endTransaction(long start) {
     assert Thread.holdsLock(this);
     
     // update statistics
-    long end = now();
+    long end = monotonicNow();
     numTransactions++;
     totalTimeTransactions += (end-start);
     if (metrics != null) // Metrics is non-null only when used inside name node
@@ -640,7 +640,7 @@ public class FSEditLog implements LogsPurgeable {
       }
       
       // do the sync
-      long start = now();
+      long start = monotonicNow();
       try {
         if (logStream != null) {
           logStream.flush();
@@ -657,7 +657,7 @@ public class FSEditLog implements LogsPurgeable {
           terminate(1, msg);
         }
       }
-      long elapsed = now() - start;
+      long elapsed = monotonicNow() - start;
   
       if (metrics != null) { // Metrics non-null only when used inside name node
         metrics.addSync(elapsed);
@@ -679,7 +679,7 @@ public class FSEditLog implements LogsPurgeable {
   // print statistics every 1 minute.
   //
   private void printStatistics(boolean force) {
-    long now = now();
+    long now = monotonicNow();
     if (lastPrintTime + 60000 > now && !force) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 51c167a..ad661ca 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
@@ -19,7 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.server.namenode.FSEditLogOp.TruncateOp;
 import static org.apache.hadoop.hdfs.server.namenode.FSImageFormat.renameReservedPathsOnUpgrade;
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.FilterInputStream;
 import java.io.IOException;
@@ -136,13 +136,13 @@ public class FSEditLogLoader {
     prog.beginStep(Phase.LOADING_EDITS, step);
     fsNamesys.writeLock();
     try {
-      long startTime = now();
+      long startTime = monotonicNow();
       FSImage.LOG.info("Start loading edits file " + edits.getName());
       long numEdits = loadEditRecords(edits, false, expectedStartingTxId,
           startOpt, recovery);
       FSImage.LOG.info("Edits file " + edits.getName() 
           + " of size " + edits.length() + " edits # " + numEdits 
-          + " loaded in " + (now()-startTime)/1000 + " seconds");
+          + " loaded in " + (monotonicNow()-startTime)/1000 + " seconds");
       return numEdits;
     } finally {
       edits.close();
@@ -177,7 +177,7 @@ public class FSEditLogLoader {
     Step step = createStartupProgressStep(in);
     prog.setTotal(Phase.LOADING_EDITS, step, numTxns);
     Counter counter = prog.getCounter(Phase.LOADING_EDITS, step);
-    long lastLogTime = now();
+    long lastLogTime = monotonicNow();
     long lastInodeId = fsNamesys.dir.getLastInodeId();
     
     try {
@@ -257,7 +257,7 @@ public class FSEditLogLoader {
           }
           // log progress
           if (op.hasTransactionId()) {
-            long now = now();
+            long now = monotonicNow();
             if (now - lastLogTime > REPLAY_TRANSACTION_LOG_INTERVAL) {
               long deltaTxId = lastAppliedTxId - expectedStartingTxId + 1;
               int percent = Math.round((float) deltaTxId / numTxns * 100);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 956a219..cce991f 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
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.DataInput;
 import java.io.DataInputStream;
@@ -309,7 +309,7 @@ public class FSImageFormat {
       StartupProgress prog = NameNode.getStartupProgress();
       Step step = new Step(StepType.INODES);
       prog.beginStep(Phase.LOADING_FSIMAGE, step);
-      long startTime = now();
+      long startTime = monotonicNow();
 
       //
       // Load in bits
@@ -441,8 +441,9 @@ public class FSImageFormat {
       imgDigest = new MD5Hash(digester.digest());
       loaded = true;
       
-      LOG.info("Image file " + curFile + " of size " + curFile.length() +
-          " bytes loaded in " + (now() - startTime)/1000 + " seconds.");
+      LOG.info("Image file " + curFile + " of size " + curFile.length()
+          + " bytes loaded in " + (monotonicNow() - startTime) / 1000
+          + " seconds.");
     }
 
   /** Update the root node's attributes */
@@ -1240,7 +1241,7 @@ public class FSImageFormat {
       prog.beginStep(Phase.SAVING_CHECKPOINT, step);
       prog.setTotal(Phase.SAVING_CHECKPOINT, step, numINodes);
       Counter counter = prog.getCounter(Phase.SAVING_CHECKPOINT, step);
-      long startTime = now();
+      long startTime = monotonicNow();
       //
       // Write out data
       //
@@ -1308,8 +1309,9 @@ public class FSImageFormat {
       // set md5 of the saved image
       savedDigest = new MD5Hash(digester.digest());
 
-      LOG.info("Image file " + newFile + " of size " + newFile.length() +
-          " bytes saved in " + (now() - startTime)/1000 + " seconds.");
+      LOG.info("Image file " + newFile + " of size " + newFile.length()
+          + " bytes saved in " + (monotonicNow() - startTime) / 1000
+          + " seconds.");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 86ea3ea..71c84b1 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
@@ -88,6 +88,7 @@ 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.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.BufferedWriter;
 import java.io.ByteArrayInputStream;
@@ -277,7 +278,6 @@ import org.apache.hadoop.util.ChunkedArrayList;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.log4j.Appender;
 import org.apache.log4j.AsyncAppender;
@@ -683,7 +683,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       namesystem.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
     }
 
-    long loadStart = now();
+    long loadStart = monotonicNow();
     try {
       namesystem.loadFSImage(startOpt);
     } catch (IOException ioe) {
@@ -691,7 +691,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       fsImage.close();
       throw ioe;
     }
-    long timeTakenToLoadFSImage = now() - loadStart;
+    long timeTakenToLoadFSImage = monotonicNow() - loadStart;
     LOG.info("Finished loading FSImage in " + timeTakenToLoadFSImage + " msecs");
     NameNodeMetrics nnMetrics = NameNode.getNameNodeMetrics();
     if (nnMetrics != null) {
@@ -5071,6 +5071,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
      * <br> >0 safe mode is on, but we are in extension period 
      */
     private long reached = -1;  
+    private long reachedTimestamp = -1;
     /** Total number of blocks. */
     int blockTotal; 
     /** Number of safe blocks. */
@@ -5171,6 +5172,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
      */
     private void enter() {
       this.reached = 0;
+      this.reachedTimestamp = 0;
     }
       
     /**
@@ -5194,6 +5196,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         NameNode.stateChangeLog.info("STATE* Safe mode is OFF"); 
       }
       reached = -1;
+      reachedTimestamp = -1;
       safeMode = null;
       final NetworkTopology nt = blockManager.getDatanodeManager().getNetworkTopology();
       NameNode.stateChangeLog.info("STATE* Network topology has "
@@ -5232,7 +5235,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         return false;
       }
 
-      if (now() - reached < extension) {
+      if (monotonicNow() - reached < extension) {
         reportStatus("STATE* Safe mode ON, in safe mode extension.", false);
         return false;
       }
@@ -5288,7 +5291,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         return;
       }
       // start monitor
-      reached = now();
+      reached = monotonicNow();
+      reachedTimestamp = now();
       if (smmthread == null) {
         smmthread = new Daemon(new SafeModeMonitor());
         smmthread.start();
@@ -5435,8 +5439,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       if (!thresholdsMet) {
         msg += "once the thresholds have been reached.";
-      } else if (reached + extension - now() > 0) {
-        msg += ("in " + (reached + extension - now()) / 1000 + " seconds.");
+      } else if (reached + extension - monotonicNow() > 0) {
+        msg += ("in " + (reached + extension - monotonicNow()) / 1000 + " seconds.");
       } else {
         msg += "soon.";
       }
@@ -5462,7 +5466,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         + ". Target blocks = " + blockThreshold + " for threshold = %" + threshold
         + ". Minimal replication = " + safeReplication + ".";
       if (reached > 0) 
-        resText += " Threshold was reached " + new Date(reached) + ".";
+        resText += " Threshold was reached " + new Date(reachedTimestamp) + ".";
       return resText;
     }
       
@@ -5941,7 +5945,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   @Metric
   public long getMillisSinceLastLoadedEdits() {
     if (isInStandbyState() && editLogTailer != null) {
-      return now() - editLogTailer.getLastLoadTimestamp();
+      return monotonicNow() - editLogTailer.getLastLoadTimeMs();
     } else {
       return 0;
     }
@@ -6983,7 +6987,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   private long getLastContact(DatanodeDescriptor alivenode) {
-    return (Time.now() - alivenode.getLastUpdate())/1000;
+    return (monotonicNow() - alivenode.getLastUpdateMonotonic())/1000;
   }
 
   private long getDfsUsed(DatanodeDescriptor alivenode) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index 0dafaae..9ce8ebc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -17,7 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -256,17 +256,17 @@ public class LeaseManager {
     }
     /** Only LeaseManager object can renew a lease */
     private void renew() {
-      this.lastUpdate = now();
+      this.lastUpdate = monotonicNow();
     }
 
     /** @return true if the Hard Limit Timer has expired */
     public boolean expiredHardLimit() {
-      return now() - lastUpdate > hardLimit;
+      return monotonicNow() - lastUpdate > hardLimit;
     }
 
     /** @return true if the Soft Limit Timer has expired */
     public boolean expiredSoftLimit() {
-      return now() - lastUpdate > softLimit;
+      return monotonicNow() - lastUpdate > softLimit;
     }
 
     /** Does this lease contain any path? */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 1d2439c..669f68a 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
@@ -289,7 +289,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
    * Check files on DFS, starting from the indicated path.
    */
   public void fsck() {
-    final long startTime = Time.now();
+    final long startTime = Time.monotonicNow();
     try {
       if(blockIds != null) {
 
@@ -357,7 +357,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
         }
 
         out.println("FSCK ended at " + new Date() + " in "
-            + (Time.now() - startTime + " milliseconds"));
+            + (Time.monotonicNow() - startTime + " milliseconds"));
 
         // If there were internal errors during the fsck operation, we want to
         // return FAILURE_STATUS, even if those errors were not immediately
@@ -383,7 +383,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       String errMsg = "Fsck on path '" + path + "' " + FAILURE_STATUS;
       LOG.warn(errMsg, e);
       out.println("FSCK ended at " + new Date() + " in "
-          + (Time.now() - startTime + " milliseconds"));
+          + (Time.monotonicNow() - startTime + " milliseconds"));
       out.println(e.getMessage());
       out.print("\n\n" + errMsg);
     } finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
index 3d72645..6b6c8d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/EditLogTailer.java
@@ -44,7 +44,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.security.SecurityUtil;
 
-import static org.apache.hadoop.util.Time.now;
+import static org.apache.hadoop.util.Time.monotonicNow;
 import static org.apache.hadoop.util.ExitUtil.terminate;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -84,7 +84,7 @@ public class EditLogTailer {
    * The last time we successfully loaded a non-zero number of edits from the
    * shared directory.
    */
-  private long lastLoadTimestamp;
+  private long lastLoadTimeMs;
 
   /**
    * How often the Standby should roll edit logs. Since the Standby only reads
@@ -105,7 +105,7 @@ public class EditLogTailer {
     this.namesystem = namesystem;
     this.editLog = namesystem.getEditLog();
     
-    lastLoadTimestamp = now();
+    lastLoadTimeMs = monotonicNow();
 
     logRollPeriodMs = conf.getInt(DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_KEY,
         DFSConfigKeys.DFS_HA_LOGROLL_PERIOD_DEFAULT) * 1000;
@@ -241,7 +241,7 @@ public class EditLogTailer {
       }
 
       if (editsLoaded > 0) {
-        lastLoadTimestamp = now();
+        lastLoadTimeMs = monotonicNow();
       }
       lastLoadedTxnId = image.getLastAppliedTxId();
     } finally {
@@ -250,10 +250,10 @@ public class EditLogTailer {
   }
 
   /**
-   * @return timestamp (in msec) of when we last loaded a non-zero number of edits.
+   * @return time in msec of when we last loaded a non-zero number of edits.
    */
-  public long getLastLoadTimestamp() {
-    return lastLoadTimestamp;
+  public long getLastLoadTimeMs() {
+    return lastLoadTimeMs;
   }
 
   /**
@@ -261,7 +261,7 @@ public class EditLogTailer {
    */
   private boolean tooLongSinceLastLoad() {
     return logRollPeriodMs >= 0 && 
-      (now() - lastLoadTimestamp) > logRollPeriodMs ;
+      (monotonicNow() - lastLoadTimeMs) > logRollPeriodMs ;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
index 2e67848..ae9612f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java
@@ -284,6 +284,7 @@ public class JsonUtil {
     m.put("cacheCapacity", datanodeinfo.getCacheCapacity());
     m.put("cacheUsed", datanodeinfo.getCacheUsed());
     m.put("lastUpdate", datanodeinfo.getLastUpdate());
+    m.put("lastUpdateMonotonic", datanodeinfo.getLastUpdateMonotonic());
     m.put("xceiverCount", datanodeinfo.getXceiverCount());
     m.put("networkLocation", datanodeinfo.getNetworkLocation());
     m.put("adminState", datanodeinfo.getAdminState().name());
@@ -379,6 +380,7 @@ public class JsonUtil {
         getLong(m, "cacheCapacity", 0l),
         getLong(m, "cacheUsed", 0l),
         getLong(m, "lastUpdate", 0l),
+        getLong(m, "lastUpdateMonotonic", 0l),
         getInt(m, "xceiverCount", 0),
         getString(m, "networkLocation", ""),
         AdminStates.valueOf(getString(m, "adminState", "NORMAL")));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index 31e5585..2966e51 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -97,6 +97,7 @@ message DatanodeInfoProto {
   optional AdminState adminState = 10 [default = NORMAL];
   optional uint64 cacheCapacity = 11 [default = 0];
   optional uint64 cacheUsed = 12 [default = 0];
+  optional uint64 lastUpdateMonotonic = 13 [default = 0];
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/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 7e7ff39..493351b 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
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hdfs;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
@@ -89,6 +90,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.VersionInfo;
 import org.apache.log4j.Level;
@@ -1019,7 +1021,7 @@ public class DFSTestUtil {
         DFSConfigKeys.DFS_DATANODE_HTTP_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_HTTPS_DEFAULT_PORT,
         DFSConfigKeys.DFS_DATANODE_IPC_DEFAULT_PORT,
-        1l, 2l, 3l, 4l, 0l, 0l, 5, 6, "local", adminState);
+        1l, 2l, 3l, 4l, 0l, 0l, 0l, 5, 6, "local", adminState);
   }
 
   public static DatanodeDescriptor getDatanodeDescriptor(String ipAddr,
@@ -1571,9 +1573,11 @@ public class DFSTestUtil {
     // the one to be in charge of the synchronization / recovery protocol.
     final DatanodeStorageInfo[] storages = ucBlock.getExpectedStorageLocations();
     DatanodeStorageInfo expectedPrimary = storages[0];
-    long mostRecentLastUpdate = expectedPrimary.getDatanodeDescriptor().getLastUpdate();
+    long mostRecentLastUpdate = expectedPrimary.getDatanodeDescriptor()
+        .getLastUpdateMonotonic();
     for (int i = 1; i < storages.length; i++) {
-      final long lastUpdate = storages[i].getDatanodeDescriptor().getLastUpdate();
+      final long lastUpdate = storages[i].getDatanodeDescriptor()
+          .getLastUpdateMonotonic();
       if (lastUpdate > mostRecentLastUpdate) {
         expectedPrimary = storages[i];
         mostRecentLastUpdate = lastUpdate;
@@ -1710,4 +1714,21 @@ public class DFSTestUtil {
     GenericTestUtils.setLogLevel(NameNode.stateChangeLog, level);
     GenericTestUtils.setLogLevel(NameNode.blockStateChangeLog, level);
   }
+
+  /**
+   * Set the datanode dead
+   */
+  public static void setDatanodeDead(DatanodeInfo dn) {
+    dn.setLastUpdate(0);
+    dn.setLastUpdateMonotonic(0);
+  }
+
+  /**
+   * Update lastUpdate and lastUpdateMonotonic with some offset.
+   */
+  public static void resetLastUpdatesWithOffset(DatanodeInfo dn, long offset) {
+    dn.setLastUpdate(Time.now() + offset);
+    dn.setLastUpdateMonotonic(Time.monotonicNow() + offset);
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
index a6cc71f..d92f49e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
@@ -2076,7 +2076,7 @@ public class MiniDFSCluster {
   public void setDataNodeDead(DatanodeID dnId) throws IOException {
     DatanodeDescriptor dnd =
         NameNodeAdapter.getDatanode(getNamesystem(), dnId);
-    dnd.setLastUpdate(0L);
+    DFSTestUtil.setDatanodeDead(dnd);
     BlockManagerTestUtil.checkHeartbeat(getNamesystem().getBlockManager());
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
index cc89852..fcc8e35 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestGetBlocks.java
@@ -132,7 +132,8 @@ public class TestGetBlocks {
       staleNodeInfo = cluster.getNameNode().getNamesystem().getBlockManager()
           .getDatanodeManager()
           .getDatanode(staleNode.getDatanodeId());
-      staleNodeInfo.setLastUpdate(Time.now() - staleInterval - 1);
+      DFSTestUtil.resetLastUpdatesWithOffset(staleNodeInfo,
+          -(staleInterval + 1));
 
       LocatedBlocks blocksAfterStale = client.getNamenode().getBlockLocations(
           fileName.toString(), 0, blockSize);
@@ -143,8 +144,7 @@ public class TestGetBlocks {
       // restart the staleNode's heartbeat
       DataNodeTestUtils.setHeartbeatsDisabledForTests(staleNode, false);
       // reset the first node as non-stale, so as to avoid two stale nodes
-      staleNodeInfo.setLastUpdate(Time.now());
-
+      DFSTestUtil.resetLastUpdatesWithOffset(staleNodeInfo, 0);
       LocatedBlock lastBlock = client.getLocatedBlocks(fileName.toString(), 0,
           Long.MAX_VALUE).getLastLocatedBlock();
       nodes = lastBlock.getLocations();
@@ -153,10 +153,10 @@ public class TestGetBlocks {
       staleNode = this.stopDataNodeHeartbeat(cluster, nodes[0].getHostName());
       assertNotNull(staleNode);
       // set the node as stale
-      cluster.getNameNode().getNamesystem().getBlockManager()
-          .getDatanodeManager()
-          .getDatanode(staleNode.getDatanodeId())
-          .setLastUpdate(Time.now() - staleInterval - 1);
+      DatanodeDescriptor dnDesc = cluster.getNameNode().getNamesystem()
+          .getBlockManager().getDatanodeManager()
+          .getDatanode(staleNode.getDatanodeId());
+      DFSTestUtil.resetLastUpdatesWithOffset(dnDesc, -(staleInterval + 1));
 
       LocatedBlock lastBlockAfterStale = client.getLocatedBlocks(
           fileName.toString(), 0, Long.MAX_VALUE).getLastLocatedBlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
index 8f2873e..e49cf58 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestInjectionForSimulatedStorage.java
@@ -59,7 +59,7 @@ public class TestInjectionForSimulatedStorage {
                                        ClientProtocol namenode,
                                        int expected, long maxWaitSec) 
                                        throws IOException {
-    long start = Time.now();
+    long start = Time.monotonicNow();
     
     //wait for all the blocks to be replicated;
     LOG.info("Checking for block replication for " + filename);
@@ -84,7 +84,7 @@ public class TestInjectionForSimulatedStorage {
                                actual + ".");
       
         if (maxWaitSec > 0 && 
-            (Time.now() - start) > (maxWaitSec * 1000)) {
+            (Time.monotonicNow() - start) > (maxWaitSec * 1000)) {
           throw new IOException("Timedout while waiting for all blocks to " +
                                 " be replicated for " + filename);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
index 5d93db4..1cf7add 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java
@@ -101,7 +101,7 @@ public class TestLease {
       // call renewLease() manually.
       // make it look like the soft limit has been exceeded.
       LeaseRenewer originalRenewer = dfs.getLeaseRenewer();
-      dfs.lastLeaseRenewal = Time.now()
+      dfs.lastLeaseRenewal = Time.monotonicNow()
       - HdfsConstants.LEASE_SOFTLIMIT_PERIOD - 1000;
       try {
         dfs.renewLease();
@@ -117,7 +117,7 @@ public class TestLease {
       }
 
       // make it look like the hard limit has been exceeded.
-      dfs.lastLeaseRenewal = Time.now()
+      dfs.lastLeaseRenewal = Time.monotonicNow()
       - HdfsConstants.LEASE_HARDLIMIT_PERIOD - 1000;
       dfs.renewLease();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
index 8ca4576a..11cbcad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRenewer.java
@@ -111,8 +111,8 @@ public class TestLeaseRenewer {
     renewer.put(fileId, mockStream, MOCK_DFSCLIENT);
 
     // Wait for lease to get renewed
-    long failTime = Time.now() + 5000;
-    while (Time.now() < failTime &&
+    long failTime = Time.monotonicNow() + 5000;
+    while (Time.monotonicNow() < failTime &&
         leaseRenewalCount.get() == 0) {
       Thread.sleep(50);
     }
@@ -193,11 +193,11 @@ public class TestLeaseRenewer {
     
     // Pretend to close the file
     renewer.closeFile(fileId, MOCK_DFSCLIENT);
-    renewer.setEmptyTime(Time.now());
+    renewer.setEmptyTime(Time.monotonicNow());
     
     // Should stop the renewer running within a few seconds
-    long failTime = Time.now() + 5000;
-    while (renewer.isRunning() && Time.now() < failTime) {
+    long failTime = Time.monotonicNow() + 5000;
+    while (renewer.isRunning() && Time.monotonicNow() < failTime) {
       Thread.sleep(50);
     }
     Assert.assertFalse(renewer.isRunning());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java
index 0163d95..23e2a7a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestParallelReadUtil.java
@@ -333,7 +333,7 @@ public class TestParallelReadUtil {
     }
 
     // Start the workers and wait
-    long starttime = Time.now();
+    long starttime = Time.monotonicNow();
     for (ReadWorker worker : workers) {
       worker.start();
     }
@@ -343,7 +343,7 @@ public class TestParallelReadUtil {
         worker.join();
       } catch (InterruptedException ignored) { }
     }
-    long endtime = Time.now();
+    long endtime = Time.monotonicNow();
 
     // Cleanup
     for (TestFileInfo testInfo : testInfoArr) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/75ead273/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
index 5351406..b702da0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReplication.java
@@ -272,7 +272,7 @@ public class TestReplication {
                                        ClientProtocol namenode,
                                        int expected, long maxWaitSec) 
                                        throws IOException {
-    long start = Time.now();
+    long start = Time.monotonicNow();
     
     //wait for all the blocks to be replicated;
     LOG.info("Checking for block replication for " + filename);
@@ -298,7 +298,7 @@ public class TestReplication {
       }
       
       if (maxWaitSec > 0 && 
-          (Time.now() - start) > (maxWaitSec * 1000)) {
+          (Time.monotonicNow() - start) > (maxWaitSec * 1000)) {
         throw new IOException("Timedout while waiting for all blocks to " +
                               " be replicated for " + filename);
       }


[47/51] [abbrv] hadoop git commit: HADOOP-11738. Fix a link of Protocol Buffers 2.5 for download in BUILDING.txt. (ozawa)

Posted by ka...@apache.org.
HADOOP-11738. Fix a link of Protocol Buffers 2.5 for download in BUILDING.txt. (ozawa)


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

Branch: refs/heads/YARN-2139
Commit: 323945b33a5bccc42e90ed619ea155503cc124b0
Parents: b351086
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed Mar 25 18:02:32 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Wed Mar 25 18:02:32 2015 +0900

----------------------------------------------------------------------
 BUILDING.txt                                    | 4 ++--
 hadoop-common-project/hadoop-common/CHANGES.txt | 3 +++
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/323945b3/BUILDING.txt
----------------------------------------------------------------------
diff --git a/BUILDING.txt b/BUILDING.txt
index c126c5e..02b8610 100644
--- a/BUILDING.txt
+++ b/BUILDING.txt
@@ -29,7 +29,7 @@ Installing required packages for clean install of Ubuntu 14.04 LTS Desktop:
 * Native libraries
   $ sudo apt-get -y install build-essential autoconf automake libtool cmake zlib1g-dev pkg-config libssl-dev
 * ProtocolBuffer 2.5.0
-  $ wget https://protobuf.googlecode.com/svn/rc/protobuf-2.5.0.tar.gz
+  $ wget https://github.com/google/protobuf/releases/download/v2.5.0/protobuf-2.5.0.tar.gz
   $ tar -zxvf protobuf-2.5.0.tar.gz
   $ cd protobuf-2.5.0.tar.gz
   $ ./configure
@@ -312,4 +312,4 @@ http://www.zlib.net/
 ----------------------------------------------------------------------------------
 Building distributions:
 
- * Build distribution with native code    : mvn package [-Pdist][-Pdocs][-Psrc][-Dtar]
\ No newline at end of file
+ * Build distribution with native code    : mvn package [-Pdist][-Pdocs][-Psrc][-Dtar]

http://git-wip-us.apache.org/repos/asf/hadoop/blob/323945b3/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 0c311df..a01a201 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1145,6 +1145,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11014. Potential resource leak in JavaKeyStoreProvider due to
     unclosed stream. (ozawa)
 
+    HADOOP-11738. Fix a link of Protocol Buffers 2.5 for download in BUILDING.txt.
+    (ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES


[29/51] [abbrv] hadoop git commit: HDFS-7956. Improve logging for DatanodeRegistration. Contributed by Plamen Jeliazkov.

Posted by ka...@apache.org.
HDFS-7956. Improve logging for DatanodeRegistration. Contributed by Plamen Jeliazkov.


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

Branch: refs/heads/YARN-2139
Commit: 970ee3fc56a68afade98017296cf9d057f225a46
Parents: 50ee8f4
Author: Plamen Jeliazkov <pl...@gmail.com>
Authored: Mon Mar 23 23:04:04 2015 -0700
Committer: Konstantin V Shvachko <sh...@apache.org>
Committed: Mon Mar 23 23:05:21 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/970ee3fc/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 3dd5fb3..3ea1346 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -777,6 +777,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7917. Use file to replace data dirs in test to simulate a disk failure.
     (Lei (Eddy) Xu via cnauroth)
 
+    HDFS-7956. Improve logging for DatanodeRegistration.
+    (Plamen Jeliazkov via shv)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/970ee3fc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
index e788137..7119738 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeRegistration.java
@@ -100,7 +100,7 @@ public class DatanodeRegistration extends DatanodeID
   @Override
   public String toString() {
     return getClass().getSimpleName()
-      + "(" + getIpAddr()
+      + "(" + super.toString()
       + ", datanodeUuid=" + getDatanodeUuid()
       + ", infoPort=" + getInfoPort()
       + ", infoSecurePort=" + getInfoSecurePort()


[18/51] [abbrv] hadoop git commit: HADOOP-11737. mockito's version in hadoop-nfs’ pom.xml shouldn't be specified. Contributed by Kengo Seki.

Posted by ka...@apache.org.
HADOOP-11737. mockito's version in hadoop-nfs’ pom.xml shouldn't be specified. Contributed by Kengo Seki.


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

Branch: refs/heads/YARN-2139
Commit: 0b9f12c847e26103bc2304cf7114e6d103264669
Parents: b375d1f
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Mon Mar 23 13:56:24 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Mon Mar 23 13:56:24 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt | 3 +++
 hadoop-common-project/hadoop-nfs/pom.xml        | 1 -
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b9f12c8/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 4cd2154..430015d 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -458,6 +458,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11447. Add a more meaningful toString method to SampleStat and 
     MutableStat. (kasha)
 
+    HADOOP-11737. mockito's version in hadoop-nfs’ pom.xml shouldn't be
+    specified. (Kengo Seki via ozawa)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0b9f12c8/hadoop-common-project/hadoop-nfs/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-nfs/pom.xml b/hadoop-common-project/hadoop-nfs/pom.xml
index 409ed75..e8156d9 100644
--- a/hadoop-common-project/hadoop-nfs/pom.xml
+++ b/hadoop-common-project/hadoop-nfs/pom.xml
@@ -55,7 +55,6 @@
     <dependency>
       <groupId>org.mockito</groupId>
       <artifactId>mockito-all</artifactId>
-      <version>1.8.5</version>
     </dependency>
     <dependency>
       <groupId>commons-logging</groupId>


[12/51] [abbrv] hadoop git commit: MAPREDUCE-6286. A typo in HistoryViewer makes some code useless, which causes counter limits are not reset correctly. Contributed by Zhihai Xu.

Posted by ka...@apache.org.
MAPREDUCE-6286. A typo in HistoryViewer makes some code useless, which causes counter limits are not reset correctly. Contributed by Zhihai Xu.


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

Branch: refs/heads/YARN-2139
Commit: 433542904aba5ddebf9bd9d299378647351eb13a
Parents: e1e0905
Author: Harsh J <ha...@cloudera.com>
Authored: Sun Mar 22 02:51:02 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Sun Mar 22 02:51:02 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                             | 4 ++++
 .../org/apache/hadoop/mapreduce/jobhistory/HistoryViewer.java    | 2 +-
 2 files changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43354290/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 76180a3..fc42941 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -286,6 +286,10 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
+    MAPREDUCE-6286. A typo in HistoryViewer makes some code useless, which
+    causes counter limits are not reset correctly.
+    (Zhihai Xu via harsh)
+
     MAPREDUCE-6213. NullPointerException caused by job history server addr not
     resolvable. (Peng Zhang via harsh)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43354290/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryViewer.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryViewer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryViewer.java
index 43b2df2..f343d7c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryViewer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/HistoryViewer.java
@@ -93,7 +93,7 @@ public class HistoryViewer {
       final Configuration jobConf = new Configuration(conf);
       try {
         jobConf.addResource(fs.open(jobConfPath), jobConfPath.toString());
-        Limits.reset(conf);
+        Limits.reset(jobConf);
       } catch (FileNotFoundException fnf) {
         if (LOG.isWarnEnabled()) {
           LOG.warn("Missing job conf in history", fnf);


[05/51] [abbrv] hadoop git commit: YARN-3356. Capacity Scheduler FiCaSchedulerApp should use ResourceUsage to track used-resources-by-label. Contributed by Wangda Tan

Posted by ka...@apache.org.
YARN-3356. Capacity Scheduler FiCaSchedulerApp should use ResourceUsage to track used-resources-by-label. Contributed by Wangda Tan


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

Branch: refs/heads/YARN-2139
Commit: 586348e4cbf197188057d6b843a6701cfffdaff3
Parents: d81109e
Author: Jian He <ji...@apache.org>
Authored: Fri Mar 20 13:54:01 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Fri Mar 20 13:54:01 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../scheduler/AbstractYarnScheduler.java        |   5 +-
 .../scheduler/AppSchedulingInfo.java            |  27 ++-
 .../server/resourcemanager/scheduler/Queue.java |  20 +++
 .../scheduler/ResourceUsage.java                |  19 ++-
 .../scheduler/SchedulerApplicationAttempt.java  |  50 +++---
 .../scheduler/SchedulerNode.java                |  14 ++
 .../scheduler/capacity/AbstractCSQueue.java     |  24 +++
 .../scheduler/capacity/LeafQueue.java           |  29 +++-
 .../scheduler/common/fica/FiCaSchedulerApp.java |  17 +-
 .../scheduler/fair/FSAppAttempt.java            |  11 +-
 .../resourcemanager/scheduler/fair/FSQueue.java |   8 +
 .../scheduler/fifo/FifoScheduler.java           |  12 +-
 .../yarn/server/resourcemanager/MockAM.java     |  24 +--
 .../capacity/TestCapacityScheduler.java         | 167 ++++++++++++++++++-
 .../scheduler/capacity/TestChildQueueOrder.java |   3 +-
 .../capacity/TestContainerAllocation.java       |  70 +-------
 .../scheduler/capacity/TestReservations.java    |   6 +-
 .../scheduler/capacity/TestUtils.java           | 129 ++++++++++++++
 19 files changed, 509 insertions(+), 129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index bbd018a..046b7b1 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -65,6 +65,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3357. Move TestFifoScheduler to FIFO package. (Rohith Sharmaks 
     via devaraj)
 
+    YARN-3356. Capacity Scheduler FiCaSchedulerApp should use ResourceUsage to
+    track used-resources-by-label. (Wangda Tan via jianhe)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.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/AbstractYarnScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
index 968a767..e1f94cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractYarnScheduler.java
@@ -358,14 +358,15 @@ public abstract class AbstractYarnScheduler
         container));
 
       // recover scheduler node
-      nodes.get(nm.getNodeID()).recoverContainer(rmContainer);
+      SchedulerNode schedulerNode = nodes.get(nm.getNodeID());
+      schedulerNode.recoverContainer(rmContainer);
 
       // recover queue: update headroom etc.
       Queue queue = schedulerAttempt.getQueue();
       queue.recoverContainer(clusterResource, schedulerAttempt, rmContainer);
 
       // recover scheduler attempt
-      schedulerAttempt.recoverContainer(rmContainer);
+      schedulerAttempt.recoverContainer(schedulerNode, rmContainer);
             
       // set master container for the current running AMContainer for this
       // attempt.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.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/AppSchedulingInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
index 1324c7d..84ebe9c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AppSchedulingInfo.java
@@ -20,8 +20,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -191,6 +189,16 @@ public class AppSchedulingInfo {
             request.getCapability());
         metrics.decrPendingResources(user, lastRequestContainers,
             lastRequestCapability);
+        
+        // update queue:
+        queue.incPendingResource(
+            request.getNodeLabelExpression(),
+            Resources.multiply(request.getCapability(),
+                request.getNumContainers()));
+        if (lastRequest != null) {
+          queue.decPendingResource(lastRequest.getNodeLabelExpression(),
+              Resources.multiply(lastRequestCapability, lastRequestContainers));
+        }
       }
     }
   }
@@ -376,6 +384,9 @@ public class AppSchedulingInfo {
     if (numOffSwitchContainers == 0) {
       checkForDeactivation();
     }
+    
+    queue.decPendingResource(offSwitchRequest.getNodeLabelExpression(),
+        offSwitchRequest.getCapability());
   }
   
   synchronized private void checkForDeactivation() {
@@ -404,6 +415,12 @@ public class AppSchedulingInfo {
             request.getCapability());
         newMetrics.incrPendingResources(user, request.getNumContainers(),
             request.getCapability());
+        
+        Resource delta = Resources.multiply(request.getCapability(),
+            request.getNumContainers()); 
+        // Update Queue
+        queue.decPendingResource(request.getNodeLabelExpression(), delta);
+        newQueue.incPendingResource(request.getNodeLabelExpression(), delta);
       }
     }
     oldMetrics.moveAppFrom(this);
@@ -423,6 +440,12 @@ public class AppSchedulingInfo {
       if (request != null) {
         metrics.decrPendingResources(user, request.getNumContainers(),
             request.getCapability());
+        
+        // Update Queue
+        queue.decPendingResource(
+            request.getNodeLabelExpression(),
+            Resources.multiply(request.getCapability(),
+                request.getNumContainers()));
       }
     }
     metrics.finishAppAttempt(applicationId, pending, user);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.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/Queue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java
index 4663a91..02003c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/Queue.java
@@ -90,4 +90,24 @@ public interface Queue {
    * @return default label expression
    */
   public String getDefaultNodeLabelExpression();
+
+  /**
+   * When new outstanding resource is asked, calling this will increase pending
+   * resource in a queue.
+   * 
+   * @param nodeLabel asked by application
+   * @param resourceToInc new resource asked
+   */
+  public void incPendingResource(String nodeLabel, Resource resourceToInc);
+  
+  /**
+   * When an outstanding resource is fulfilled or canceled, calling this will
+   * decrease pending resource in a queue.
+   * 
+   * @param nodeLabel
+   *          asked by application
+   * @param resourceToDec
+   *          new resource asked
+   */
+  public void decPendingResource(String nodeLabel, Resource resourceToDec);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.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/ResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
index de44bbe..36ee4da 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/ResourceUsage.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler;
 
 import java.util.HashMap;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
@@ -75,14 +76,17 @@ public class ResourceUsage {
       };
     }
     
+    public Resource getUsed() {
+      return resArr[ResourceType.USED.idx];
+    }
+
     @Override
     public String toString() {
       StringBuilder sb = new StringBuilder();
       sb.append("{used=" + resArr[0] + "%, ");
       sb.append("pending=" + resArr[1] + "%, ");
       sb.append("am_used=" + resArr[2] + "%, ");
-      sb.append("reserved=" + resArr[3] + "%, ");
-      sb.append("headroom=" + resArr[4] + "%}");
+      sb.append("reserved=" + resArr[3] + "%}");
       return sb.toString();
     }
   }
@@ -117,6 +121,17 @@ public class ResourceUsage {
   public void setUsed(Resource res) {
     setUsed(NL, res);
   }
+  
+  public void copyAllUsed(ResourceUsage other) {
+    try {
+      writeLock.lock();
+      for (Entry<String, UsageByLabel> entry : other.usages.entrySet()) {
+        setUsed(entry.getKey(), Resources.clone(entry.getValue().getUsed()));
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
 
   public void setUsed(String label, Resource res) {
     _set(label, ResourceType.USED, res);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.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/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index 9816699..799a5c1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -87,13 +87,12 @@ public class SchedulerApplicationAttempt {
 
   private final Multiset<Priority> reReservations = HashMultiset.create();
   
-  protected final Resource currentReservation = Resource.newInstance(0, 0);
   private Resource resourceLimit = Resource.newInstance(0, 0);
-  protected Resource currentConsumption = Resource.newInstance(0, 0);
-  private Resource amResource = Resources.none();
   private boolean unmanagedAM = true;
   private boolean amRunning = false;
   private LogAggregationContext logAggregationContext;
+  
+  protected ResourceUsage attemptResourceUsage = new ResourceUsage();
 
   protected List<RMContainer> newlyAllocatedContainers = 
       new ArrayList<RMContainer>();
@@ -217,11 +216,11 @@ public class SchedulerApplicationAttempt {
   }
   
   public Resource getAMResource() {
-    return amResource;
+    return attemptResourceUsage.getAMUsed();
   }
 
   public void setAMResource(Resource amResource) {
-    this.amResource = amResource;
+    attemptResourceUsage.setAMUsed(amResource);
   }
 
   public boolean isAmRunning() {
@@ -260,7 +259,7 @@ public class SchedulerApplicationAttempt {
   @Stable
   @Private
   public synchronized Resource getCurrentReservation() {
-    return currentReservation;
+    return attemptResourceUsage.getReserved();
   }
   
   public Queue getQueue() {
@@ -311,8 +310,8 @@ public class SchedulerApplicationAttempt {
       rmContainer = 
           new RMContainerImpl(container, getApplicationAttemptId(), 
               node.getNodeID(), appSchedulingInfo.getUser(), rmContext);
-        
-      Resources.addTo(currentReservation, container.getResource());
+      attemptResourceUsage.incReserved(node.getPartition(),
+          container.getResource());
       
       // Reset the re-reservation count
       resetReReservations(priority);
@@ -336,7 +335,7 @@ public class SchedulerApplicationAttempt {
           + " reserved container " + rmContainer + " on node " + node
           + ". This attempt currently has " + reservedContainers.size()
           + " reserved containers at priority " + priority
-          + "; currentReservation " + currentReservation.getMemory());
+          + "; currentReservation " + container.getResource());
     }
 
     return rmContainer;
@@ -402,9 +401,9 @@ public class SchedulerApplicationAttempt {
       for (Priority priority : getPriorities()) {
         Map<String, ResourceRequest> requests = getResourceRequests(priority);
         if (requests != null) {
-          LOG.debug("showRequests:" + " application=" + getApplicationId() + 
-              " headRoom=" + getHeadroom() + 
-              " currentConsumption=" + currentConsumption.getMemory());
+          LOG.debug("showRequests:" + " application=" + getApplicationId()
+              + " headRoom=" + getHeadroom() + " currentConsumption="
+              + attemptResourceUsage.getUsed().getMemory());
           for (ResourceRequest request : requests.values()) {
             LOG.debug("showRequests:" + " application=" + getApplicationId()
                 + " request=" + request);
@@ -415,7 +414,7 @@ public class SchedulerApplicationAttempt {
   }
   
   public Resource getCurrentConsumption() {
-    return currentConsumption;
+    return attemptResourceUsage.getUsed();
   }
 
   public static class ContainersAndNMTokensAllocation {
@@ -548,12 +547,17 @@ public class SchedulerApplicationAttempt {
   }
 
   public synchronized ApplicationResourceUsageReport getResourceUsageReport() {
-    AggregateAppResourceUsage resUsage = getRunningAggregateAppResourceUsage();
+    AggregateAppResourceUsage runningResourceUsage =
+        getRunningAggregateAppResourceUsage();
+    Resource usedResourceClone =
+        Resources.clone(attemptResourceUsage.getUsed());
+    Resource reservedResourceClone =
+        Resources.clone(attemptResourceUsage.getReserved());
     return ApplicationResourceUsageReport.newInstance(liveContainers.size(),
-               reservedContainers.size(), Resources.clone(currentConsumption),
-               Resources.clone(currentReservation),
-               Resources.add(currentConsumption, currentReservation),
-               resUsage.getMemorySeconds(), resUsage.getVcoreSeconds());
+        reservedContainers.size(), usedResourceClone, reservedResourceClone,
+        Resources.add(usedResourceClone, reservedResourceClone),
+        runningResourceUsage.getMemorySeconds(),
+        runningResourceUsage.getVcoreSeconds());
   }
 
   public synchronized Map<ContainerId, RMContainer> getLiveContainersMap() {
@@ -572,7 +576,7 @@ public class SchedulerApplicationAttempt {
       SchedulerApplicationAttempt appAttempt) {
     this.liveContainers = appAttempt.getLiveContainersMap();
     // this.reReservations = appAttempt.reReservations;
-    this.currentConsumption = appAttempt.getCurrentConsumption();
+    this.attemptResourceUsage.copyAllUsed(appAttempt.attemptResourceUsage);
     this.resourceLimit = appAttempt.getResourceLimit();
     // this.currentReservation = appAttempt.currentReservation;
     // this.newlyAllocatedContainers = appAttempt.newlyAllocatedContainers;
@@ -603,7 +607,8 @@ public class SchedulerApplicationAttempt {
     this.queue = newQueue;
   }
 
-  public synchronized void recoverContainer(RMContainer rmContainer) {
+  public synchronized void recoverContainer(SchedulerNode node,
+      RMContainer rmContainer) {
     // recover app scheduling info
     appSchedulingInfo.recoverContainer(rmContainer);
 
@@ -613,8 +618,9 @@ public class SchedulerApplicationAttempt {
     LOG.info("SchedulerAttempt " + getApplicationAttemptId()
       + " is recovering container " + rmContainer.getContainerId());
     liveContainers.put(rmContainer.getContainerId(), rmContainer);
-    Resources.addTo(currentConsumption, rmContainer.getContainer()
-      .getResource());
+    attemptResourceUsage.incUsed(node.getPartition(), rmContainer
+        .getContainer().getResource());
+    
     // resourceLimit: updated when LeafQueue#recoverContainer#allocateResource
     // is called.
     // newlyAllocatedContainers.add(rmContainer);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.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/SchedulerNode.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
index 2901134..f03663a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerNode.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
 import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
@@ -294,4 +295,17 @@ public abstract class SchedulerNode {
   public void updateLabels(Set<String> labels) {
     this.labels = labels;
   }
+  
+  /**
+   * Get partition of which the node belongs to, if node-labels of this node is
+   * empty or null, it belongs to NO_LABEL partition. And since we only support
+   * one partition for each node (YARN-2694), first label will be its partition.
+   */
+  public String getPartition() {
+    if (this.labels == null || this.labels.isEmpty()) {
+      return RMNodeLabelsManager.NO_LABEL; 
+    } else {
+      return this.labels.iterator().next();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.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/AbstractCSQueue.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/AbstractCSQueue.java
index 4e53060..3cd85ae 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/AbstractCSQueue.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/AbstractCSQueue.java
@@ -509,4 +509,28 @@ public abstract class AbstractCSQueue implements CSQueue {
     // non-empty
     return false;
   }
+  
+  @Override
+  public void incPendingResource(String nodeLabel, Resource resourceToInc) {
+    if (nodeLabel == null) {
+      nodeLabel = RMNodeLabelsManager.NO_LABEL;
+    }
+    // ResourceUsage has its own lock, no addition lock needs here.
+    queueUsage.incPending(nodeLabel, resourceToInc);
+    if (null != parent) {
+      parent.incPendingResource(nodeLabel, resourceToInc);
+    }
+  }
+  
+  @Override
+  public void decPendingResource(String nodeLabel, Resource resourceToDec) {
+    if (nodeLabel == null) {
+      nodeLabel = RMNodeLabelsManager.NO_LABEL;
+    }
+    // ResourceUsage has its own lock, no addition lock needs here.
+    queueUsage.decPending(nodeLabel, resourceToDec);
+    if (null != parent) {
+      parent.decPendingResource(nodeLabel, resourceToDec);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/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 fa0e280..3e5405d 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
@@ -739,6 +739,15 @@ public class LeafQueue extends AbstractCSQueue {
     return labels;
   }
   
+  private boolean checkResourceRequestMatchingNodeLabel(ResourceRequest offswitchResourceRequest,
+      FiCaSchedulerNode node) {
+    String askedNodeLabel = offswitchResourceRequest.getNodeLabelExpression();
+    if (null == askedNodeLabel) {
+      askedNodeLabel = RMNodeLabelsManager.NO_LABEL;
+    }
+    return askedNodeLabel.equals(node.getPartition());
+  }
+  
   @Override
   public synchronized CSAssignment assignContainers(Resource clusterResource,
       FiCaSchedulerNode node, ResourceLimits currentResourceLimits) {
@@ -796,6 +805,14 @@ public class LeafQueue extends AbstractCSQueue {
           if (application.getTotalRequiredResources(priority) <= 0) {
             continue;
           }
+          
+          // Is the node-label-expression of this offswitch resource request
+          // matches the node's label?
+          // If not match, jump to next priority.
+          if (!checkResourceRequestMatchingNodeLabel(anyRequest, node)) {
+            continue;
+          }
+          
           if (!this.reservationsContinueLooking) {
             if (!shouldAllocOrReserveNewContainer(application, priority, required)) {
               if (LOG.isDebugEnabled()) {
@@ -825,7 +842,7 @@ public class LeafQueue extends AbstractCSQueue {
           }
 
           // Check user limit
-          if (!assignToUser(clusterResource, application.getUser(), userLimit,
+          if (!canAssignToUser(clusterResource, application.getUser(), userLimit,
               application, true, requestedNodeLabels)) {
             break;
           }
@@ -1076,7 +1093,7 @@ public class LeafQueue extends AbstractCSQueue {
   }
   
   @Private
-  protected synchronized boolean assignToUser(Resource clusterResource,
+  protected synchronized boolean canAssignToUser(Resource clusterResource,
       String userName, Resource limit, FiCaSchedulerApp application,
       boolean checkReservations, Set<String> requestLabels) {
     User user = getUser(userName);
@@ -1094,7 +1111,8 @@ public class LeafQueue extends AbstractCSQueue {
             limit)) {
       // if enabled, check to see if could we potentially use this node instead
       // of a reserved node if the application has reserved containers
-      if (this.reservationsContinueLooking && checkReservations) {
+      if (this.reservationsContinueLooking && checkReservations
+          && label.equals(CommonNodeLabelsManager.NO_LABEL)) {
         if (Resources.lessThanOrEqual(
             resourceCalculator,
             clusterResource,
@@ -1305,7 +1323,7 @@ public class LeafQueue extends AbstractCSQueue {
     }
 
     // Check user limit
-    if (!assignToUser(clusterResource, application.getUser(), userLimit,
+    if (!canAssignToUser(clusterResource, application.getUser(), userLimit,
         application, false, null)) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("was going to reserve but hit user limit");
@@ -1622,7 +1640,8 @@ public class LeafQueue extends AbstractCSQueue {
               node, rmContainer);
         } else {
           removed =
-            application.containerCompleted(rmContainer, containerStatus, event);
+              application.containerCompleted(rmContainer, containerStatus,
+                  event, node.getPartition());
           node.releaseContainer(container);
         }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.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/common/fica/FiCaSchedulerApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
index 10f5c20..e041389 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/common/fica/FiCaSchedulerApp.java
@@ -90,7 +90,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
   }
 
   synchronized public boolean containerCompleted(RMContainer rmContainer,
-      ContainerStatus containerStatus, RMContainerEventType event) {
+      ContainerStatus containerStatus, RMContainerEventType event,
+      String partition) {
 
     // Remove from the list of containers
     if (null == liveContainers.remove(rmContainer.getContainerId())) {
@@ -122,7 +123,7 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     // Update usage metrics 
     Resource containerResource = rmContainer.getContainer().getResource();
     queue.getMetrics().releaseResources(getUser(), 1, containerResource);
-    Resources.subtractFrom(currentConsumption, containerResource);
+    attemptResourceUsage.decUsed(partition, containerResource);
 
     // Clear resource utilization metrics cache.
     lastMemoryAggregateAllocationUpdateTime = -1;
@@ -156,7 +157,8 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
     // Update consumption and track allocations
     List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
         type, node, priority, request, container);
-    Resources.addTo(currentConsumption, container.getResource());
+    attemptResourceUsage.incUsed(node.getPartition(),
+        container.getResource());
     
     // Update resource requests related to "request" and store in RMContainer 
     ((RMContainerImpl)rmContainer).setResourceRequests(resourceRequestList);
@@ -198,12 +200,13 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
         resetReReservations(priority);
 
         Resource resource = reservedContainer.getContainer().getResource();
-        Resources.subtractFrom(currentReservation, resource);
+        this.attemptResourceUsage.decReserved(node.getPartition(), resource);
 
         LOG.info("Application " + getApplicationId() + " unreserved "
-            + " on node " + node + ", currently has " + reservedContainers.size()
-            + " at priority " + priority + "; currentReservation "
-            + currentReservation);
+            + " on node " + node + ", currently has "
+            + reservedContainers.size() + " at priority " + priority
+            + "; currentReservation " + this.attemptResourceUsage.getReserved()
+            + " on node-label=" + node.getPartition());
         return true;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.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/fair/FSAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
index 67103d1..dfde5ab 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSAppAttempt.java
@@ -142,7 +142,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     // Update usage metrics 
     Resource containerResource = rmContainer.getContainer().getResource();
     queue.getMetrics().releaseResources(getUser(), 1, containerResource);
-    Resources.subtractFrom(currentConsumption, containerResource);
+    this.attemptResourceUsage.decUsed(containerResource);
 
     // remove from preemption map if it is completed
     preemptionMap.remove(rmContainer);
@@ -164,11 +164,12 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     resetReReservations(priority);
 
     Resource resource = reservedContainer.getContainer().getResource();
-    Resources.subtractFrom(currentReservation, resource);
+    this.attemptResourceUsage.decReserved(resource);
 
     LOG.info("Application " + getApplicationId() + " unreserved " + " on node "
-        + node + ", currently has " + reservedContainers.size() + " at priority "
-        + priority + "; currentReservation " + currentReservation);
+        + node + ", currently has " + reservedContainers.size()
+        + " at priority " + priority + "; currentReservation "
+        + this.attemptResourceUsage.getReserved());
   }
 
   @Override
@@ -339,7 +340,7 @@ public class FSAppAttempt extends SchedulerApplicationAttempt
     // Update consumption and track allocations
     List<ResourceRequest> resourceRequestList = appSchedulingInfo.allocate(
         type, node, priority, request, container);
-    Resources.addTo(currentConsumption, container.getResource());
+    this.attemptResourceUsage.incUsed(container.getResource());
 
     // Update resource requests related to "request" and store in RMContainer
     ((RMContainerImpl) rmContainer).setResourceRequests(resourceRequestList);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.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/fair/FSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
index 349464e..1562bf6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FSQueue.java
@@ -291,4 +291,12 @@ public abstract class FSQueue implements Queue, Schedulable {
     // TODO, add implementation for FS
     return null;
   }
+  
+  @Override
+  public void incPendingResource(String nodeLabel, Resource resourceToInc) {
+  }
+  
+  @Override
+  public void decPendingResource(String nodeLabel, Resource resourceToDec) {
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.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/fifo/FifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
index beb3ab5..b8c419c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoScheduler.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
+import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppEventType;
@@ -200,6 +201,14 @@ public class FifoScheduler extends
       // TODO add implementation for FIFO scheduler
       return null;
     }
+
+    @Override
+    public void incPendingResource(String nodeLabel, Resource resourceToInc) {
+    }
+
+    @Override
+    public void decPendingResource(String nodeLabel, Resource resourceToDec) {
+    }
   };
 
   public FifoScheduler() {
@@ -870,7 +879,8 @@ public class FifoScheduler extends
     }
 
     // Inform the application
-    application.containerCompleted(rmContainer, containerStatus, event);
+    application.containerCompleted(rmContainer, containerStatus, event,
+        RMNodeLabelsManager.NO_LABEL);
 
     // Inform the node
     node.releaseContainer(container);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.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/MockAM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
index 494f5a4..f62fdb3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
@@ -164,17 +164,19 @@ public class MockAM {
   public List<ResourceRequest> createReq(String[] hosts, int memory, int priority,
       int containers, String labelExpression) throws Exception {
     List<ResourceRequest> reqs = new ArrayList<ResourceRequest>();
-    for (String host : hosts) {
-      // only add host/rack request when asked host isn't ANY
-      if (!host.equals(ResourceRequest.ANY)) {
-        ResourceRequest hostReq =
-            createResourceReq(host, memory, priority, containers,
-                labelExpression);
-        reqs.add(hostReq);
-        ResourceRequest rackReq =
-            createResourceReq("/default-rack", memory, priority, containers,
-                labelExpression);
-        reqs.add(rackReq);
+    if (hosts != null) {
+      for (String host : hosts) {
+        // only add host/rack request when asked host isn't ANY
+        if (!host.equals(ResourceRequest.ANY)) {
+          ResourceRequest hostReq =
+              createResourceReq(host, memory, priority, containers,
+                  labelExpression);
+          reqs.add(hostReq);
+          ResourceRequest rackReq =
+              createResourceReq("/default-rack", memory, priority, containers,
+                  labelExpression);
+          reqs.add(rackReq);
+        }
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.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/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
index e30f441..aaa615d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
@@ -29,11 +29,13 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.CyclicBarrier;
 
@@ -111,6 +113,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaS
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptAddedSchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppAttemptRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
@@ -128,10 +131,13 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
-import org.junit.ComparisonFailure;
 import org.junit.Test;
 import org.mockito.Mockito;
 
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
 
 public class TestCapacityScheduler {
   private static final Log LOG = LogFactory.getLog(TestCapacityScheduler.class);
@@ -2557,6 +2563,165 @@ public class TestCapacityScheduler {
     Assert.fail("Shouldn't successfully allocate containers for am2, "
         + "queue-a's max capacity will be violated if container allocated");
   }
+  
+  @SuppressWarnings("unchecked")
+  private <E> Set<E> toSet(E... elements) {
+    Set<E> set = Sets.newHashSet(elements);
+    return set;
+  }
+  
+  @Test
+  public void testQueueHierarchyPendingResourceUpdate() throws Exception {
+    Configuration conf =
+        TestUtils.getConfigurationWithQueueLabels(new Configuration(false));
+    conf.setBoolean(YarnConfiguration.NODE_LABELS_ENABLED, true);
+    
+    final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
+    mgr.init(conf);
+    mgr.addToCluserNodeLabels(ImmutableSet.of("x", "y"));
+    mgr.addLabelsToNode(ImmutableMap.of(NodeId.newInstance("h1", 0), toSet("x")));
+    
+    MemoryRMStateStore memStore = new MemoryRMStateStore();
+    memStore.init(conf);
+    MockRM rm = new MockRM(conf, memStore) {
+      protected RMNodeLabelsManager createNodeLabelManager() {
+        return mgr;
+      }
+    };
+    
+    rm.start();
+    MockNM nm1 = // label = x
+        new MockNM("h1:1234", 200 * GB, rm.getResourceTrackerService());
+    nm1.registerNode();
+    
+    MockNM nm2 = // label = ""
+        new MockNM("h2:1234", 200 * GB, rm.getResourceTrackerService());
+    nm2.registerNode();
+    
+    // Launch app1 in queue=a1
+    RMApp app1 = rm.submitApp(1 * GB, "app", "user", null, "a1");
+    MockAM am1 = MockRM.launchAndRegisterAM(app1, rm, nm2);
+    
+    // Launch app2 in queue=b1  
+    RMApp app2 = rm.submitApp(8 * GB, "app", "user", null, "b1");
+    MockAM am2 = MockRM.launchAndRegisterAM(app2, rm, nm2);
+    
+    // am1 asks for 8 * 1GB container for no label
+    am1.allocate(Arrays.asList(ResourceRequest.newInstance(
+        Priority.newInstance(1), "*", Resources.createResource(1 * GB), 8)),
+        null);
+    
+    checkPendingResource(rm, "a1", 8 * GB, null);
+    checkPendingResource(rm, "a", 8 * GB, null);
+    checkPendingResource(rm, "root", 8 * GB, null);
+    
+    // am2 asks for 8 * 1GB container for no label
+    am2.allocate(Arrays.asList(ResourceRequest.newInstance(
+        Priority.newInstance(1), "*", Resources.createResource(1 * GB), 8)),
+        null);
+    
+    checkPendingResource(rm, "a1", 8 * GB, null);
+    checkPendingResource(rm, "a", 8 * GB, null);
+    checkPendingResource(rm, "b1", 8 * GB, null);
+    checkPendingResource(rm, "b", 8 * GB, null);
+    // root = a + b
+    checkPendingResource(rm, "root", 16 * GB, null);
+    
+    // am2 asks for 8 * 1GB container in another priority for no label
+    am2.allocate(Arrays.asList(ResourceRequest.newInstance(
+        Priority.newInstance(2), "*", Resources.createResource(1 * GB), 8)),
+        null);
+    
+    checkPendingResource(rm, "a1", 8 * GB, null);
+    checkPendingResource(rm, "a", 8 * GB, null);
+    checkPendingResource(rm, "b1", 16 * GB, null);
+    checkPendingResource(rm, "b", 16 * GB, null);
+    // root = a + b
+    checkPendingResource(rm, "root", 24 * GB, null);
+    
+    // am1 asks 4 GB resource instead of 8 * GB for priority=1
+    am1.allocate(Arrays.asList(ResourceRequest.newInstance(
+        Priority.newInstance(1), "*", Resources.createResource(4 * GB), 1)),
+        null);
+    
+    checkPendingResource(rm, "a1", 4 * GB, null);
+    checkPendingResource(rm, "a", 4 * GB, null);
+    checkPendingResource(rm, "b1", 16 * GB, null);
+    checkPendingResource(rm, "b", 16 * GB, null);
+    // root = a + b
+    checkPendingResource(rm, "root", 20 * GB, null);
+    
+    // am1 asks 8 * GB resource which label=x
+    am1.allocate(Arrays.asList(ResourceRequest.newInstance(
+        Priority.newInstance(2), "*", Resources.createResource(8 * GB), 1,
+        true, "x")), null);
+    
+    checkPendingResource(rm, "a1", 4 * GB, null);
+    checkPendingResource(rm, "a", 4 * GB, null);
+    checkPendingResource(rm, "a1", 8 * GB, "x");
+    checkPendingResource(rm, "a", 8 * GB, "x");
+    checkPendingResource(rm, "b1", 16 * GB, null);
+    checkPendingResource(rm, "b", 16 * GB, null);
+    // root = a + b
+    checkPendingResource(rm, "root", 20 * GB, null);
+    checkPendingResource(rm, "root", 8 * GB, "x");
+    
+    // some containers allocated for am1, pending resource should decrease
+    ContainerId containerId =
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
+    Assert.assertTrue(rm.waitForState(nm1, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    containerId = ContainerId.newContainerId(am1.getApplicationAttemptId(), 3);
+    Assert.assertTrue(rm.waitForState(nm2, containerId,
+        RMContainerState.ALLOCATED, 10 * 1000));
+    
+    checkPendingResource(rm, "a1", 0 * GB, null);
+    checkPendingResource(rm, "a", 0 * GB, null);
+    checkPendingResource(rm, "a1", 0 * GB, "x");
+    checkPendingResource(rm, "a", 0 * GB, "x");
+    // some containers could be allocated for am2 when we allocating containers
+    // for am1, just check if pending resource of b1/b/root > 0 
+    checkPendingResourceGreaterThanZero(rm, "b1", null);
+    checkPendingResourceGreaterThanZero(rm, "b", null);
+    // root = a + b
+    checkPendingResourceGreaterThanZero(rm, "root", null);
+    checkPendingResource(rm, "root", 0 * GB, "x");
+    
+    // complete am2, pending resource should be 0 now
+    AppAttemptRemovedSchedulerEvent appRemovedEvent =
+        new AppAttemptRemovedSchedulerEvent(
+          am2.getApplicationAttemptId(), RMAppAttemptState.FINISHED, false);
+    rm.getResourceScheduler().handle(appRemovedEvent);
+    
+    checkPendingResource(rm, "a1", 0 * GB, null);
+    checkPendingResource(rm, "a", 0 * GB, null);
+    checkPendingResource(rm, "a1", 0 * GB, "x");
+    checkPendingResource(rm, "a", 0 * GB, "x"); 
+    checkPendingResource(rm, "b1", 0 * GB, null);
+    checkPendingResource(rm, "b", 0 * GB, null);
+    checkPendingResource(rm, "root", 0 * GB, null);
+    checkPendingResource(rm, "root", 0 * GB, "x");
+  }
+  
+  private void checkPendingResource(MockRM rm, String queueName, int memory,
+      String label) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    CSQueue queue = cs.getQueue(queueName);
+    Assert.assertEquals(
+        memory,
+        queue.getQueueResourceUsage()
+            .getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
+            .getMemory());
+  }
+
+  private void checkPendingResourceGreaterThanZero(MockRM rm, String queueName,
+      String label) {
+    CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler();
+    CSQueue queue = cs.getQueue(queueName);
+    Assert.assertTrue(queue.getQueueResourceUsage()
+        .getPending(label == null ? RMNodeLabelsManager.NO_LABEL : label)
+        .getMemory() > 0);
+  }
 
   // Test verifies AM Used resource for LeafQueue when AM ResourceRequest is
   // lesser than minimumAllocation

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.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/scheduler/capacity/TestChildQueueOrder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
index 71dc523..23b31fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestChildQueueOrder.java
@@ -247,7 +247,8 @@ public class TestChildQueueOrder {
     // Stub an App and its containerCompleted
     FiCaSchedulerApp app_0 = getMockApplication(0,user_0);
     doReturn(true).when(app_0).containerCompleted(any(RMContainer.class),
-        any(ContainerStatus.class),any(RMContainerEventType.class));
+        any(ContainerStatus.class), any(RMContainerEventType.class),
+        any(String.class));
 
     Priority priority = TestUtils.createMockPriority(1); 
     ContainerAllocationExpirer expirer = 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.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/scheduler/capacity/TestContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
index fe61eab..03b8f5c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
@@ -328,19 +328,6 @@ public class TestContainerAllocation {
     MockRM.launchAndRegisterAM(app1, rm1, nm1);
   }
   
-  private Configuration getConfigurationWithDefaultQueueLabels(
-      Configuration config) {
-    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
-    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
-    
-    CapacitySchedulerConfiguration conf =
-        (CapacitySchedulerConfiguration) getConfigurationWithQueueLabels(config);
-        new CapacitySchedulerConfiguration(config);
-    conf.setDefaultNodeLabelExpression(A, "x");
-    conf.setDefaultNodeLabelExpression(B, "y");
-    return conf;
-  }
-  
   private Configuration getConfigurationWithQueueLabels(Configuration config) {
     CapacitySchedulerConfiguration conf =
         new CapacitySchedulerConfiguration(config);
@@ -406,57 +393,6 @@ public class TestContainerAllocation {
     return set;
   }
   
-  private Configuration getComplexConfigurationWithQueueLabels(
-      Configuration config) {
-    CapacitySchedulerConfiguration conf =
-        new CapacitySchedulerConfiguration(config);
-    
-    // Define top-level queues
-    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
-    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
-    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100);
-    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "z", 100);
-
-    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
-    conf.setCapacity(A, 10);
-    conf.setMaximumCapacity(A, 10);
-    conf.setAccessibleNodeLabels(A, toSet("x", "y"));
-    conf.setCapacityByLabel(A, "x", 100);
-    conf.setCapacityByLabel(A, "y", 50);
-    
-    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
-    conf.setCapacity(B, 90);
-    conf.setMaximumCapacity(B, 100);
-    conf.setAccessibleNodeLabels(B, toSet("y", "z"));
-    conf.setCapacityByLabel(B, "y", 50);
-    conf.setCapacityByLabel(B, "z", 100);
-    
-    // Define 2nd-level queues
-    final String A1 = A + ".a1";
-    conf.setQueues(A, new String[] {"a1"});
-    conf.setCapacity(A1, 100);
-    conf.setMaximumCapacity(A1, 100);
-    conf.setAccessibleNodeLabels(A1, toSet("x", "y"));
-    conf.setDefaultNodeLabelExpression(A1, "x");
-    conf.setCapacityByLabel(A1, "x", 100);
-    conf.setCapacityByLabel(A1, "y", 100);
-    
-    conf.setQueues(B, new String[] {"b1", "b2"});
-    final String B1 = B + ".b1";
-    conf.setCapacity(B1, 50);
-    conf.setMaximumCapacity(B1, 50);
-    conf.setAccessibleNodeLabels(B1, RMNodeLabelsManager.EMPTY_STRING_SET);
-
-    final String B2 = B + ".b2";
-    conf.setCapacity(B2, 50);
-    conf.setMaximumCapacity(B2, 50);
-    conf.setAccessibleNodeLabels(B2, toSet("y", "z"));
-    conf.setCapacityByLabel(B2, "y", 100);
-    conf.setCapacityByLabel(B2, "z", 100);
-
-    return conf;
-  }
-  
   @Test (timeout = 300000)
   public void testContainerAllocationWithSingleUserLimits() throws Exception {
     final RMNodeLabelsManager mgr = new NullRMNodeLabelsManager();
@@ -468,7 +404,7 @@ public class TestContainerAllocation {
         NodeId.newInstance("h2", 0), toSet("y")));
 
     // inject node label manager
-    MockRM rm1 = new MockRM(getConfigurationWithDefaultQueueLabels(conf)) {
+    MockRM rm1 = new MockRM(TestUtils.getConfigurationWithDefaultQueueLabels(conf)) {
       @Override
       public RMNodeLabelsManager createNodeLabelManager() {
         return mgr;
@@ -554,7 +490,7 @@ public class TestContainerAllocation {
         RMNodeLabelsManager.EMPTY_STRING_SET));
 
     // inject node label manager
-    MockRM rm1 = new MockRM(getComplexConfigurationWithQueueLabels(conf)) {
+    MockRM rm1 = new MockRM(TestUtils.getComplexConfigurationWithQueueLabels(conf)) {
       @Override
       public RMNodeLabelsManager createNodeLabelManager() {
         return mgr;
@@ -711,7 +647,7 @@ public class TestContainerAllocation {
         NodeId.newInstance("h2", 0), toSet("y")));
 
     // inject node label manager
-    MockRM rm1 = new MockRM(getConfigurationWithDefaultQueueLabels(conf)) {
+    MockRM rm1 = new MockRM(TestUtils.getConfigurationWithDefaultQueueLabels(conf)) {
       @Override
       public RMNodeLabelsManager createNodeLabelManager() {
         return mgr;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.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/scheduler/capacity/TestReservations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
index c5b7587..e8a8243 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestReservations.java
@@ -1058,19 +1058,19 @@ public class TestReservations {
 
     // set limit so subtrace reservations it can continue
     Resource limit = Resources.createResource(12 * GB, 0);
-    boolean res = a.assignToUser(clusterResource, user_0, limit, app_0,
+    boolean res = a.canAssignToUser(clusterResource, user_0, limit, app_0,
         true, null);
     assertTrue(res);
 
     // tell it not to check for reservations and should fail as already over
     // limit
-    res = a.assignToUser(clusterResource, user_0, limit, app_0, false, null);
+    res = a.canAssignToUser(clusterResource, user_0, limit, app_0, false, null);
     assertFalse(res);
 
     refreshQueuesTurnOffReservationsContLook(a, csConf);
 
     // should now return false since feature off
-    res = a.assignToUser(clusterResource, user_0, limit, app_0, true, null);
+    res = a.canAssignToUser(clusterResource, user_0, limit, app_0, true, null);
     assertFalse(res);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/586348e4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.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/scheduler/capacity/TestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
index 9e352a7..62135b9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java
@@ -60,6 +60,8 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
+import com.google.common.collect.Sets;
+
 public class TestUtils {
   private static final Log LOG = LogFactory.getLog(TestUtils.class);
 
@@ -216,4 +218,131 @@ public class TestUtils {
     when(container.getPriority()).thenReturn(priority);
     return container;
   }
+  
+  @SuppressWarnings("unchecked")
+  private static <E> Set<E> toSet(E... elements) {
+    Set<E> set = Sets.newHashSet(elements);
+    return set;
+  }
+  
+  /**
+   * Get a queue structure:
+   * <pre>
+   *             Root
+   *            /  |  \
+   *           a   b   c
+   *           |   |   |
+   *           a1  b1  c1
+   *          (x)  (y)
+   * </pre>  
+   */
+  public static Configuration getConfigurationWithQueueLabels(Configuration config) {
+    CapacitySchedulerConfiguration conf =
+        new CapacitySchedulerConfiguration(config);
+    
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b", "c"});
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100);
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 15);
+    conf.setAccessibleNodeLabels(A, toSet("x"));
+    conf.setCapacityByLabel(A, "x", 100);
+    
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 20);
+    conf.setAccessibleNodeLabels(B, toSet("y"));
+    conf.setCapacityByLabel(B, "y", 100);
+    
+    final String C = CapacitySchedulerConfiguration.ROOT + ".c";
+    conf.setCapacity(C, 70);
+    conf.setMaximumCapacity(C, 70);
+    conf.setAccessibleNodeLabels(C, RMNodeLabelsManager.EMPTY_STRING_SET);
+    
+    // Define 2nd-level queues
+    final String A1 = A + ".a1";
+    conf.setQueues(A, new String[] {"a1"});
+    conf.setCapacity(A1, 100);
+    conf.setMaximumCapacity(A1, 100);
+    conf.setCapacityByLabel(A1, "x", 100);
+    
+    final String B1 = B + ".b1";
+    conf.setQueues(B, new String[] {"b1"});
+    conf.setCapacity(B1, 100);
+    conf.setMaximumCapacity(B1, 100);
+    conf.setCapacityByLabel(B1, "y", 100);
+
+    final String C1 = C + ".c1";
+    conf.setQueues(C, new String[] {"c1"});
+    conf.setCapacity(C1, 100);
+    conf.setMaximumCapacity(C1, 100);
+    
+    return conf;
+  }
+  
+  public static Configuration getComplexConfigurationWithQueueLabels(
+      Configuration config) {
+    CapacitySchedulerConfiguration conf =
+        new CapacitySchedulerConfiguration(config);
+    
+    // Define top-level queues
+    conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[] {"a", "b"});
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "x", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "y", 100);
+    conf.setCapacityByLabel(CapacitySchedulerConfiguration.ROOT, "z", 100);
+
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    conf.setCapacity(A, 10);
+    conf.setMaximumCapacity(A, 10);
+    conf.setAccessibleNodeLabels(A, toSet("x", "y"));
+    conf.setCapacityByLabel(A, "x", 100);
+    conf.setCapacityByLabel(A, "y", 50);
+    
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    conf.setCapacity(B, 90);
+    conf.setMaximumCapacity(B, 100);
+    conf.setAccessibleNodeLabels(B, toSet("y", "z"));
+    conf.setCapacityByLabel(B, "y", 50);
+    conf.setCapacityByLabel(B, "z", 100);
+    
+    // Define 2nd-level queues
+    final String A1 = A + ".a1";
+    conf.setQueues(A, new String[] {"a1"});
+    conf.setCapacity(A1, 100);
+    conf.setMaximumCapacity(A1, 100);
+    conf.setAccessibleNodeLabels(A1, toSet("x", "y"));
+    conf.setDefaultNodeLabelExpression(A1, "x");
+    conf.setCapacityByLabel(A1, "x", 100);
+    conf.setCapacityByLabel(A1, "y", 100);
+    
+    conf.setQueues(B, new String[] {"b1", "b2"});
+    final String B1 = B + ".b1";
+    conf.setCapacity(B1, 50);
+    conf.setMaximumCapacity(B1, 50);
+    conf.setAccessibleNodeLabels(B1, RMNodeLabelsManager.EMPTY_STRING_SET);
+
+    final String B2 = B + ".b2";
+    conf.setCapacity(B2, 50);
+    conf.setMaximumCapacity(B2, 50);
+    conf.setAccessibleNodeLabels(B2, toSet("y", "z"));
+    conf.setCapacityByLabel(B2, "y", 100);
+    conf.setCapacityByLabel(B2, "z", 100);
+
+    return conf;
+  }
+  
+  public static Configuration getConfigurationWithDefaultQueueLabels(
+      Configuration config) {
+    final String A = CapacitySchedulerConfiguration.ROOT + ".a";
+    final String B = CapacitySchedulerConfiguration.ROOT + ".b";
+    
+    CapacitySchedulerConfiguration conf =
+        (CapacitySchedulerConfiguration) getConfigurationWithQueueLabels(config);
+        new CapacitySchedulerConfiguration(config);
+    conf.setDefaultNodeLabelExpression(A, "x");
+    conf.setDefaultNodeLabelExpression(B, "y");
+    return conf;
+  }
 }


[06/51] [abbrv] hadoop git commit: HADOOP-11447. Add a more meaningful toString method to SampleStat and MutableStat. (kasha)

Posted by ka...@apache.org.
HADOOP-11447. Add a more meaningful toString method to SampleStat and MutableStat. (kasha)


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

Branch: refs/heads/YARN-2139
Commit: fe5c23b670c773145b87fecfaf9191536e9f1c51
Parents: 586348e
Author: Karthik Kambatla <ka...@apache.org>
Authored: Fri Mar 20 17:03:03 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Fri Mar 20 17:03:03 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt        |  3 +++
 .../org/apache/hadoop/metrics2/lib/MutableStat.java    |  4 ++++
 .../org/apache/hadoop/metrics2/util/SampleStat.java    | 13 +++++++++++++
 3 files changed, 20 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5c23b6/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 823a36b..4cd2154 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -455,6 +455,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11709. Time.NANOSECONDS_PER_MILLISECOND - use class-level final
     constant instead of method variable (Ajith S via ozawa)
 
+    HADOOP-11447. Add a more meaningful toString method to SampleStat and 
+    MutableStat. (kasha)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5c23b6/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableStat.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableStat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableStat.java
index ba37757..d794e8e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableStat.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableStat.java
@@ -151,4 +151,8 @@ public class MutableStat extends MutableMetric {
     minMax.reset();
   }
 
+  @Override
+  public String toString() {
+    return lastStat().toString();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fe5c23b6/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/SampleStat.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/SampleStat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/SampleStat.java
index 589062a..cd9aaa4 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/SampleStat.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/SampleStat.java
@@ -137,6 +137,19 @@ public class SampleStat {
     return minmax.max();
   }
 
+  @Override
+  public String toString() {
+    try {
+      return "Samples = " + numSamples() +
+          "  Min = " + min() +
+          "  Mean = " + mean() +
+          "  Std Dev = " + stddev() +
+          "  Max = " + max();
+    } catch (Throwable t) {
+      return super.toString();
+    }
+  }
+
   /**
    * Helper to keep running min/max
    */


[14/51] [abbrv] hadoop git commit: MAPREDUCE-5190. Unnecessary condition test in RandomSampler. Contributed by Jingguo Yao.

Posted by ka...@apache.org.
MAPREDUCE-5190. Unnecessary condition test in RandomSampler. Contributed by Jingguo Yao.


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

Branch: refs/heads/YARN-2139
Commit: 1d5c796d654c8959972d15cc6742731a99380bfc
Parents: b46c2bb
Author: Harsh J <ha...@cloudera.com>
Authored: Sun Mar 22 10:03:25 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Sun Mar 22 10:03:25 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                           | 3 +++
 .../apache/hadoop/mapreduce/lib/partition/InputSampler.java    | 6 ++----
 2 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d5c796d/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 2920811..e98aacd 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -256,6 +256,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-5190. Unnecessary condition test in RandomSampler.
+    (Jingguo Yao via harsh)
+
     MAPREDUCE-6239. Consolidate TestJobConf classes in
     hadoop-mapreduce-client-jobclient and hadoop-mapreduce-client-core
     (Varun Saxena via harsh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/1d5c796d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/InputSampler.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/InputSampler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/InputSampler.java
index 4668f49..cce9f37 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/InputSampler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/InputSampler.java
@@ -230,10 +230,8 @@ public class InputSampler<K,V> extends Configured implements Tool  {
               // to reflect the possibility of existing elements being
               // pushed out
               int ind = r.nextInt(numSamples);
-              if (ind != numSamples) {
-                samples.set(ind, ReflectionUtils.copy(job.getConfiguration(),
-                                 reader.getCurrentKey(), null));
-              }
+              samples.set(ind, ReflectionUtils.copy(job.getConfiguration(),
+                               reader.getCurrentKey(), null));
               freq *= (numSamples - 1) / (double) numSamples;
             }
           }


[30/51] [abbrv] hadoop git commit: YARN-1880. Cleanup TestApplicationClientProtocolOnHA. Contributed by ozawa.

Posted by ka...@apache.org.
YARN-1880. Cleanup TestApplicationClientProtocolOnHA. Contributed by ozawa.


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

Branch: refs/heads/YARN-2139
Commit: fbceb3b41834d6899c4353fb24f12ba3ecf67faf
Parents: 970ee3f
Author: Harsh J <ha...@cloudera.com>
Authored: Tue Mar 24 11:57:28 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Tue Mar 24 11:57:58 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                     |  3 +++
 .../client/TestApplicationClientProtocolOnHA.java   | 16 ++++++++++------
 2 files changed, 13 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbceb3b4/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 3d9f271..8a5e142 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -58,6 +58,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    YARN-1880. Cleanup TestApplicationClientProtocolOnHA
+    (ozawa via harsh)
+
     YARN-3243. CapacityScheduler should pass headroom from parent to children
     to make sure ParentQueue obey its capacity limits. (Wangda Tan via jianhe)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fbceb3b4/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationClientProtocolOnHA.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationClientProtocolOnHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationClientProtocolOnHA.java
index bfc6656..8e00554 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationClientProtocolOnHA.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestApplicationClientProtocolOnHA.java
@@ -93,7 +93,8 @@ public class TestApplicationClientProtocolOnHA extends ProtocolHATestBase {
   public void testGetApplicationsOnHA() throws Exception {
     List<ApplicationReport> reports =
         client.getApplications();
-    Assert.assertTrue(reports != null && !reports.isEmpty());
+    Assert.assertTrue(reports != null);
+    Assert.assertFalse(reports.isEmpty());
     Assert.assertEquals(cluster.createFakeAppReports(),
         reports);
   }
@@ -101,7 +102,8 @@ public class TestApplicationClientProtocolOnHA extends ProtocolHATestBase {
   @Test(timeout = 15000)
   public void testGetClusterNodesOnHA() throws Exception {
     List<NodeReport> reports = client.getNodeReports(NodeState.RUNNING);
-    Assert.assertTrue(reports != null && !reports.isEmpty());
+    Assert.assertTrue(reports != null);
+    Assert.assertFalse(reports.isEmpty());
     Assert.assertEquals(cluster.createFakeNodeReports(),
         reports);
   }
@@ -117,8 +119,8 @@ public class TestApplicationClientProtocolOnHA extends ProtocolHATestBase {
   @Test(timeout = 15000)
   public void testGetQueueUserAclsOnHA() throws Exception {
     List<QueueUserACLInfo> queueUserAclsList = client.getQueueAclsInfo();
-    Assert.assertTrue(queueUserAclsList != null
-        && !queueUserAclsList.isEmpty());
+    Assert.assertTrue(queueUserAclsList != null);
+    Assert.assertFalse(queueUserAclsList.isEmpty());
     Assert.assertEquals(cluster.createFakeQueueUserACLInfoList(),
         queueUserAclsList);
   }
@@ -136,7 +138,8 @@ public class TestApplicationClientProtocolOnHA extends ProtocolHATestBase {
   public void testGetApplicationAttemptsOnHA() throws Exception {
     List<ApplicationAttemptReport> reports =
         client.getApplicationAttempts(cluster.createFakeAppId());
-    Assert.assertTrue(reports != null && !reports.isEmpty());
+    Assert.assertTrue(reports != null);
+    Assert.assertFalse(reports.isEmpty());
     Assert.assertEquals(cluster.createFakeApplicationAttemptReports(),
         reports);
   }
@@ -153,7 +156,8 @@ public class TestApplicationClientProtocolOnHA extends ProtocolHATestBase {
   public void testGetContainersOnHA() throws Exception {
     List<ContainerReport> reports =
         client.getContainers(cluster.createFakeApplicationAttemptId());
-    Assert.assertTrue(reports != null && !reports.isEmpty());
+    Assert.assertTrue(reports != null);
+    Assert.assertFalse(reports.isEmpty());
     Assert.assertEquals(cluster.createFakeContainerReports(),
         reports);
   }


[49/51] [abbrv] hadoop git commit: MAPREDUCE-6292. Use org.junit package instead of junit.framework in TestCombineFileInputFormat. (aajisaka)

Posted by ka...@apache.org.
MAPREDUCE-6292. Use org.junit package instead of junit.framework in TestCombineFileInputFormat. (aajisaka)


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

Branch: refs/heads/YARN-2139
Commit: c770df49b4c9d5ac5ddaa4ff2ec541e4bd1a8e56
Parents: b6dea97
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Mar 25 19:00:35 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Mar 25 19:00:35 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../lib/input/TestCombineFileInputFormat.java   | 36 ++++++++++++--------
 2 files changed, 25 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c770df49/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index f81a13f..9d6f1d4 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -389,6 +389,9 @@ Release 2.7.0 - UNRELEASED
     MAPREDUCE-6265. Make ContainerLauncherImpl.INITIAL_POOL_SIZE configurable 
     to better control to launch/kill containers. (Zhihai Xu via ozawa)
 
+    MAPREDUCE-6292. Use org.junit package instead of junit.framework in
+    TestCombineFileInputFormat. (aajisaka)
+
   OPTIMIZATIONS
 
     MAPREDUCE-6169. MergeQueue should release reference to the current item 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c770df49/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
index db51ec6..85c675c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
@@ -29,9 +29,6 @@ import java.util.TreeMap;
 import java.util.concurrent.TimeoutException;
 import java.util.zip.GZIPOutputStream;
 
-import org.junit.Assert;
-import junit.framework.TestCase;
-
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -60,7 +57,11 @@ import org.junit.Test;
 
 import com.google.common.collect.HashMultiset;
 
-public class TestCombineFileInputFormat extends TestCase {
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestCombineFileInputFormat {
 
   private static final String rack1[] = new String[] {
     "/r1"
@@ -221,6 +222,7 @@ public class TestCombineFileInputFormat extends TestCase {
     }
   }
 
+  @Test
   public void testRecordReaderInit() throws InterruptedException, IOException {
     // Test that we properly initialize the child recordreader when
     // CombineFileInputFormat and CombineFileRecordReader are used.
@@ -258,6 +260,7 @@ public class TestCombineFileInputFormat extends TestCase {
       rr.getCurrentKey().toString());
   }
 
+  @Test
   public void testReinit() throws Exception {
     // Test that a split containing multiple files works correctly,
     // with the child RecordReader getting its initialize() method
@@ -296,6 +299,7 @@ public class TestCombineFileInputFormat extends TestCase {
     assertFalse(rr.nextKeyValue());
   }
 
+  @Test
   public void testSplitPlacement() throws Exception {
     MiniDFSCluster dfs = null;
     FileSystem fileSys = null;
@@ -725,6 +729,7 @@ public class TestCombineFileInputFormat extends TestCase {
     DFSTestUtil.waitReplication(fileSys, name, replication);
   }
 
+  @Test
   public void testNodeDistribution() throws IOException, InterruptedException {
     DummyInputFormat inFormat = new DummyInputFormat();
     int numBlocks = 60;
@@ -774,20 +779,21 @@ public class TestCombineFileInputFormat extends TestCase {
         maxSplitSize, minSizeNode, minSizeRack, splits);
 
     int expectedSplitCount = (int) (totLength / maxSplitSize);
-    Assert.assertEquals(expectedSplitCount, splits.size());
+    assertEquals(expectedSplitCount, splits.size());
 
     // Ensure 90+% of the splits have node local blocks.
     // 100% locality may not always be achieved.
     int numLocalSplits = 0;
     for (InputSplit inputSplit : splits) {
-      Assert.assertEquals(maxSplitSize, inputSplit.getLength());
+      assertEquals(maxSplitSize, inputSplit.getLength());
       if (inputSplit.getLocations().length == 1) {
         numLocalSplits++;
       }
     }
-    Assert.assertTrue(numLocalSplits >= 0.9 * splits.size());
+    assertTrue(numLocalSplits >= 0.9 * splits.size());
   }
-  
+
+  @Test
   public void testNodeInputSplit() throws IOException, InterruptedException {
     // Regression test for MAPREDUCE-4892. There are 2 nodes with all blocks on 
     // both nodes. The grouping ensures that both nodes get splits instead of 
@@ -826,18 +832,19 @@ public class TestCombineFileInputFormat extends TestCase {
                           maxSize, minSizeNode, minSizeRack, splits);
     
     int expectedSplitCount = (int)(totLength/maxSize);
-    Assert.assertEquals(expectedSplitCount, splits.size());
+    assertEquals(expectedSplitCount, splits.size());
     HashMultiset<String> nodeSplits = HashMultiset.create();
     for(int i=0; i<expectedSplitCount; ++i) {
       InputSplit inSplit = splits.get(i);
-      Assert.assertEquals(maxSize, inSplit.getLength());
-      Assert.assertEquals(1, inSplit.getLocations().length);
+      assertEquals(maxSize, inSplit.getLength());
+      assertEquals(1, inSplit.getLocations().length);
       nodeSplits.add(inSplit.getLocations()[0]);
     }
-    Assert.assertEquals(3, nodeSplits.count(locations[0]));
-    Assert.assertEquals(3, nodeSplits.count(locations[1]));
+    assertEquals(3, nodeSplits.count(locations[0]));
+    assertEquals(3, nodeSplits.count(locations[1]));
   }
-  
+
+  @Test
   public void testSplitPlacementForCompressedFiles() throws Exception {
     MiniDFSCluster dfs = null;
     FileSystem fileSys = null;
@@ -1190,6 +1197,7 @@ public class TestCombineFileInputFormat extends TestCase {
   /**
    * Test that CFIF can handle missing blocks.
    */
+  @Test
   public void testMissingBlocks() throws Exception {
     String namenode = null;
     MiniDFSCluster dfs = null;


[27/51] [abbrv] hadoop git commit: HDFS-7884. Fix NullPointerException in BlockSender when the generation stamp provided by the client is larger than the one stored in the datanode. Contributed by Brahma Reddy Battula

Posted by ka...@apache.org.
HDFS-7884. Fix NullPointerException in BlockSender when the generation stamp provided by the client is larger than the one stored in the datanode.  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/d7e3c336
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d7e3c336
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d7e3c336

Branch: refs/heads/YARN-2139
Commit: d7e3c3364eb904f55a878bc14c331952f9dadab2
Parents: 9fae455
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Tue Mar 24 13:49:17 2015 +0900
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Tue Mar 24 13:49:17 2015 +0900

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7e3c336/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 b88b7e3..d2891e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1237,6 +1237,10 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7942. NFS: support regexp grouping in nfs.exports.allowed.hosts (brandonli)
 
+    HDFS-7884. Fix NullPointerException in BlockSender when the generation stamp
+    provided by the client is larger than the one stored in the datanode.
+    (Brahma Reddy Battula via szetszwo)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d7e3c336/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index f4cde11..e76b93a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -246,6 +246,13 @@ class BlockSender implements java.io.Closeable {
       if (replica.getGenerationStamp() < block.getGenerationStamp()) {
         throw new IOException("Replica gen stamp < block genstamp, block="
             + block + ", replica=" + replica);
+      } else if (replica.getGenerationStamp() > block.getGenerationStamp()) {
+        if (DataNode.LOG.isDebugEnabled()) {
+          DataNode.LOG.debug("Bumping up the client provided"
+              + " block's genstamp to latest " + replica.getGenerationStamp()
+              + " for block " + block);
+        }
+        block.setGenerationStamp(replica.getGenerationStamp());
       }
       if (replicaVisibleLength < 0) {
         throw new IOException("Replica is not readable, block="


[11/51] [abbrv] hadoop git commit: MAPREDUCE-6213. NullPointerException caused by job history server addr not resolvable. Contributed by Peng Zhang.

Posted by ka...@apache.org.
MAPREDUCE-6213. NullPointerException caused by job history server addr not resolvable. Contributed by Peng Zhang.


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

Branch: refs/heads/YARN-2139
Commit: e1e09052e861926112493d6041aae01ab594b547
Parents: 7a678db
Author: Harsh J <ha...@cloudera.com>
Authored: Sun Mar 22 02:44:36 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Sun Mar 22 02:44:36 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                          | 3 +++
 .../org/apache/hadoop/mapreduce/v2/util/MRWebAppUtil.java     | 7 ++++---
 2 files changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1e09052/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 4f80411..76180a3 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -286,6 +286,9 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
+    MAPREDUCE-6213. NullPointerException caused by job history server addr not
+    resolvable. (Peng Zhang via harsh)
+
     MAPREDUCE-6281. Fix javadoc in Terasort. (Albert Chu via ozawa)
 
 Release 2.7.0 - UNRELEASED

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1e09052/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRWebAppUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRWebAppUtil.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRWebAppUtil.java
index cac0119..d367060 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRWebAppUtil.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRWebAppUtil.java
@@ -137,8 +137,9 @@ public class MRWebAppUtil {
       hsAddress, getDefaultJHSWebappPort(),
       getDefaultJHSWebappURLWithoutScheme());
     StringBuffer sb = new StringBuffer();
-    if (address.getAddress().isAnyLocalAddress() || 
-        address.getAddress().isLoopbackAddress()) {
+    if (address.getAddress() != null &&
+        (address.getAddress().isAnyLocalAddress() ||
+         address.getAddress().isLoopbackAddress())) {
       sb.append(InetAddress.getLocalHost().getCanonicalHostName());
     } else {
       sb.append(address.getHostName());
@@ -171,4 +172,4 @@ public class MRWebAppUtil {
   public static String getAMWebappScheme(Configuration conf) {
     return "http://";
   }
-}
\ No newline at end of file
+}


[26/51] [abbrv] hadoop git commit: YARN-3393. Getting application(s) goes wrong when app finishes before starting the attempt. Contributed by Zhijie Shen

Posted by ka...@apache.org.
YARN-3393. Getting application(s) goes wrong when app finishes before
starting the attempt. Contributed by Zhijie Shen


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

Branch: refs/heads/YARN-2139
Commit: 9fae455e26e0230107e1c6db58a49a5b6b296cf4
Parents: 2c238ae
Author: Xuan <xg...@apache.org>
Authored: Mon Mar 23 20:33:16 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Mon Mar 23 20:33:16 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 ...pplicationHistoryManagerOnTimelineStore.java | 13 +++----
 ...pplicationHistoryManagerOnTimelineStore.java | 39 +++++++++++++++++---
 3 files changed, 42 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fae455e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e7d4f59..3d9f271 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -828,6 +828,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3336. FileSystem memory leak in DelegationTokenRenewer.
     (Zhihai Xu via cnauroth)
 
+    YARN-3393. Getting application(s) goes wrong when app finishes before
+    starting the attempt. (Zhijie Shen via xgong)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fae455e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
index 1010f62..49041c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryManagerOnTimelineStore.java
@@ -517,15 +517,14 @@ public class ApplicationHistoryManagerOnTimelineStore extends AbstractService
       if (app.appReport.getCurrentApplicationAttemptId() != null) {
         ApplicationAttemptReport appAttempt =
             getApplicationAttempt(app.appReport.getCurrentApplicationAttemptId());
-        if (appAttempt != null) {
-          app.appReport.setHost(appAttempt.getHost());
-          app.appReport.setRpcPort(appAttempt.getRpcPort());
-          app.appReport.setTrackingUrl(appAttempt.getTrackingUrl());
-          app.appReport.setOriginalTrackingUrl(appAttempt.getOriginalTrackingUrl());
-        }
+        app.appReport.setHost(appAttempt.getHost());
+        app.appReport.setRpcPort(appAttempt.getRpcPort());
+        app.appReport.setTrackingUrl(appAttempt.getTrackingUrl());
+        app.appReport.setOriginalTrackingUrl(appAttempt.getOriginalTrackingUrl());
       }
-    } catch (AuthorizationException e) {
+    } catch (AuthorizationException | ApplicationAttemptNotFoundException e) {
       // AuthorizationException is thrown because the user doesn't have access
+      // It's possible that the app is finished before the first attempt is created.
       app.appReport.setDiagnostics(null);
       app.appReport.setCurrentApplicationAttemptId(null);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9fae455e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
index 50a15f1..8cf1240 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
@@ -76,6 +76,10 @@ public class TestApplicationHistoryManagerOnTimelineStore {
   @BeforeClass
   public static void prepareStore() throws Exception {
     store = createStore(SCALE);
+    TimelineEntities entities = new TimelineEntities();
+    entities.addEntity(createApplicationTimelineEntity(
+        ApplicationId.newInstance(0, SCALE + 1), true, false));
+    store.put(entities);
   }
 
   public static TimelineStore createStore(int scale) throws Exception {
@@ -129,9 +133,9 @@ public class TestApplicationHistoryManagerOnTimelineStore {
       TimelineEntities entities = new TimelineEntities();
       ApplicationId appId = ApplicationId.newInstance(0, i);
       if (i == 2) {
-        entities.addEntity(createApplicationTimelineEntity(appId, true));
+        entities.addEntity(createApplicationTimelineEntity(appId, true, true));
       } else {
-        entities.addEntity(createApplicationTimelineEntity(appId, false));
+        entities.addEntity(createApplicationTimelineEntity(appId, false, true));
       }
       store.put(entities);
       for (int j = 1; j <= scale; ++j) {
@@ -216,6 +220,27 @@ public class TestApplicationHistoryManagerOnTimelineStore {
   }
 
   @Test
+  public void testGetApplicationReportWithNotAttempt() throws Exception {
+    final ApplicationId appId = ApplicationId.newInstance(0, SCALE + 1);
+    ApplicationReport app;
+    if (callerUGI == null) {
+      app = historyManager.getApplication(appId);
+    } else {
+      app =
+          callerUGI.doAs(new PrivilegedExceptionAction<ApplicationReport> () {
+            @Override
+            public ApplicationReport run() throws Exception {
+              return historyManager.getApplication(appId);
+            }
+          });
+    }
+    Assert.assertNotNull(app);
+    Assert.assertEquals(appId, app.getApplicationId());
+    Assert.assertEquals(ApplicationAttemptId.newInstance(appId, -1),
+        app.getCurrentApplicationAttemptId());
+  }
+
+  @Test
   public void testGetApplicationAttemptReport() throws Exception {
     final ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1);
@@ -308,7 +333,7 @@ public class TestApplicationHistoryManagerOnTimelineStore {
     Collection<ApplicationReport> apps =
         historyManager.getAllApplications().values();
     Assert.assertNotNull(apps);
-    Assert.assertEquals(SCALE, apps.size());
+    Assert.assertEquals(SCALE + 1, apps.size());
   }
 
   @Test
@@ -408,7 +433,7 @@ public class TestApplicationHistoryManagerOnTimelineStore {
   }
 
   private static TimelineEntity createApplicationTimelineEntity(
-      ApplicationId appId, boolean emptyACLs) {
+      ApplicationId appId, boolean emptyACLs, boolean noAttempt) {
     TimelineEntity entity = new TimelineEntity();
     entity.setEntityType(ApplicationMetricsConstants.ENTITY_TYPE);
     entity.setEntityId(appId.toString());
@@ -447,8 +472,10 @@ public class TestApplicationHistoryManagerOnTimelineStore {
         FinalApplicationStatus.UNDEFINED.toString());
     eventInfo.put(ApplicationMetricsConstants.STATE_EVENT_INFO,
         YarnApplicationState.FINISHED.toString());
-    eventInfo.put(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO,
-        ApplicationAttemptId.newInstance(appId, 1));
+    if (noAttempt) {
+      eventInfo.put(ApplicationMetricsConstants.LATEST_APP_ATTEMPT_EVENT_INFO,
+          ApplicationAttemptId.newInstance(appId, 1));
+    }
     tEvent.setEventInfo(eventInfo);
     entity.addEvent(tEvent);
     return entity;


[16/51] [abbrv] hadoop git commit: MAPREDUCE-5448. Addendum fix to remove deprecation warning by junit.Assert import in TestFileOutputCommitter.

Posted by ka...@apache.org.
MAPREDUCE-5448. Addendum fix to remove deprecation warning by junit.Assert import in TestFileOutputCommitter.


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

Branch: refs/heads/YARN-2139
Commit: 4cd54d9a297435150ab61803284eb05603f114e2
Parents: 8770c82
Author: Harsh J <ha...@cloudera.com>
Authored: Sun Mar 22 10:33:15 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Sun Mar 22 10:33:15 2015 +0530

----------------------------------------------------------------------
 .../hadoop/mapreduce/lib/output/TestFileOutputCommitter.java      | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4cd54d9a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java
index 5c4428b..7678f35 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java
@@ -27,7 +27,6 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
-import junit.framework.Assert;
 import junit.framework.TestCase;
 
 import org.apache.commons.logging.Log;
@@ -315,7 +314,7 @@ public class TestFileOutputCommitter extends TestCase {
     try {
       MapFileOutputFormat.getReaders(outDir, conf);
     } catch (Exception e) {
-      Assert.fail("Fail to read from MapFileOutputFormat: " + e);
+      fail("Fail to read from MapFileOutputFormat: " + e);
       e.printStackTrace();
     }
 


[20/51] [abbrv] hadoop git commit: HDFS-7942. NFS: support regexp grouping in nfs.exports.allowed.hosts. Contributed by Brandon Li

Posted by ka...@apache.org.
HDFS-7942. NFS: support regexp grouping in nfs.exports.allowed.hosts. Contributed by Brandon Li


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

Branch: refs/heads/YARN-2139
Commit: 36af4a913c97113bd0486c48e1cb864c5cba46fd
Parents: 82eda77
Author: Brandon Li <br...@apache.org>
Authored: Mon Mar 23 10:06:47 2015 -0700
Committer: Brandon Li <br...@apache.org>
Committed: Mon Mar 23 10:06:47 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/nfs/NfsExports.java  |  2 +-
 .../org/apache/hadoop/nfs/TestNfsExports.java   | 22 ++++++++++++++++++--
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 ++
 .../src/site/markdown/HdfsNfsGateway.md         |  8 ++++---
 4 files changed, 28 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/36af4a91/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java
index 8b6b46a..af96565 100644
--- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java
+++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/NfsExports.java
@@ -391,7 +391,7 @@ public class NfsExports {
       return new CIDRMatch(privilege,
           new SubnetUtils(pair[0], pair[1]).getInfo());
     } else if (host.contains("*") || host.contains("?") || host.contains("[")
-        || host.contains("]")) {
+        || host.contains("]") || host.contains("(") || host.contains(")")) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Using Regex match for '" + host + "' and " + privilege);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/36af4a91/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java b/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java
index 349e82a..542975d 100644
--- a/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java
+++ b/hadoop-common-project/hadoop-nfs/src/test/java/org/apache/hadoop/nfs/TestNfsExports.java
@@ -23,8 +23,8 @@ import org.junit.Test;
 
 public class TestNfsExports {
 
-  private final String address1 = "192.168.0.1";
-  private final String address2 = "10.0.0.1";
+  private final String address1 = "192.168.0.12";
+  private final String address2 = "10.0.0.12";
   private final String hostname1 = "a.b.com";
   private final String hostname2 = "a.b.org";
   
@@ -165,6 +165,24 @@ public class TestNfsExports {
   }
   
   @Test
+  public void testRegexGrouping() {
+    NfsExports matcher = new NfsExports(CacheSize, ExpirationPeriod,
+        "192.168.0.(12|34)");
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname1));
+    // address1 will hit the cache
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege(address1, hostname2));
+
+    matcher = new NfsExports(CacheSize, ExpirationPeriod, "\\w*.a.b.com");
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege("1.2.3.4", "web.a.b.com"));
+    // address "1.2.3.4" will hit the cache
+    Assert.assertEquals(AccessPrivilege.READ_ONLY,
+        matcher.getAccessPrivilege("1.2.3.4", "email.a.b.org"));
+  }
+  
+  @Test
   public void testMultiMatchers() throws Exception {
     long shortExpirationPeriod = 1 * 1000 * 1000 * 1000; // 1s
     NfsExports matcher = new NfsExports(CacheSize, shortExpirationPeriod, 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/36af4a91/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 e82c4c4..8c99876 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1232,6 +1232,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-6841. Use Time.monotonicNow() wherever applicable instead of Time.now()
     (Vinayakumar B via kihwal)
 
+    HDFS-7942. NFS: support regexp grouping in nfs.exports.allowed.hosts (brandonli)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/36af4a91/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 e6666d4..b7e1733 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
@@ -144,10 +144,12 @@ It's strongly recommended for the users to update a few configuration properties
 *   By default, the export can be mounted by any client. To better control the access,
     users can update the following property. The value string contains machine name and
     access privilege, separated by whitespace
-    characters. The machine name format can be a single host, a Java regular expression, or an IPv4 address. The access
+    characters. The machine name format can be a single host, a "*", a Java regular expression, or an IPv4 address. The access
     privilege uses rw or ro to specify read/write or read-only access of the machines to exports. If the access privilege is not provided, the default is read-only. Entries are separated by ";".
-    For example: "192.168.0.0/22 rw ; host.\*\\.example\\.com ; host1.test.org ro;". Only the NFS gateway needs to restart after
-    this property is updated.
+    For example: "192.168.0.0/22 rw ; \\\\w\*\\\\.example\\\\.com ; host1.test.org ro;". Only the NFS gateway needs to restart after
+    this property is updated. Note that, here Java regular expression is differnt with the regrulation expression used in 
+    Linux NFS export table, such as, using "\\\\w\*\\\\.example\\\\.com" instead of "\*.example.com", "192\\\\.168\\\\.0\\\\.(11|22)"
+    instead of "192.168.0.[11|22]" and so on.  
 
         <property>
           <name>nfs.exports.allowed.hosts</name>


[39/51] [abbrv] hadoop git commit: HDFS-7977. NFS couldn't take percentile intervals. Contributed by Brandon Li

Posted by ka...@apache.org.
HDFS-7977. NFS couldn't take percentile intervals. Contributed by Brandon Li


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

Branch: refs/heads/YARN-2139
Commit: 570a83ae80faf2076966acf30588733803327844
Parents: 97a7277
Author: Brandon Li <br...@apache.org>
Authored: Tue Mar 24 10:49:16 2015 -0700
Committer: Brandon Li <br...@apache.org>
Committed: Tue Mar 24 10:49:16 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java |  1 -
 .../org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Metrics.java   | 13 ++++++-------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt            |  2 ++
 .../hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md    | 12 ++++++++++++
 4 files changed, 20 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/570a83ae/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java
index 09ee579..05cc0b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/conf/NfsConfigKeys.java
@@ -72,7 +72,6 @@ public class NfsConfigKeys {
   public static final String NFS_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:" + NFS_HTTPS_PORT_DEFAULT;
   
   public static final String  NFS_METRICS_PERCENTILES_INTERVALS_KEY = "nfs.metrics.percentiles.intervals";
-  public static final String  NFS_METRICS_PERCENTILES_INTERVALS_DEFAULT = "";
   
   /*
    * HDFS super-user is the user with the same identity as NameNode process

http://git-wip-us.apache.org/repos/asf/hadoop/blob/570a83ae/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Metrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Metrics.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Metrics.java
index d36ea73..880a8a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Metrics.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Metrics.java
@@ -90,9 +90,9 @@ public class Nfs3Metrics {
       readNanosQuantiles[i] = registry.newQuantiles("readProcessNanos"
           + interval + "s", "Read process in ns", "ops", "latency", interval);
       writeNanosQuantiles[i] = registry.newQuantiles("writeProcessNanos"
-          + interval + "s", " process in ns", "ops", "latency", interval);
+          + interval + "s", "Write process in ns", "ops", "latency", interval);
       commitNanosQuantiles[i] = registry.newQuantiles("commitProcessNanos"
-          + interval + "s", "Read process in ns", "ops", "latency", interval);
+          + interval + "s", "Commit process in ns", "ops", "latency", interval);
     }
   }
 
@@ -101,10 +101,9 @@ public class Nfs3Metrics {
     MetricsSystem ms = DefaultMetricsSystem.instance();
     JvmMetrics jm = JvmMetrics.create(gatewayName, sessionId, ms);
 
-    // Percentile measurement is [,,,] by default 
-    int[] intervals = conf.getInts(conf.get(
-        NfsConfigKeys.NFS_METRICS_PERCENTILES_INTERVALS_KEY,
-        NfsConfigKeys.NFS_METRICS_PERCENTILES_INTERVALS_DEFAULT));
+    // Percentile measurement is [50th,75th,90th,95th,99th] currently 
+    int[] intervals = conf
+        .getInts(NfsConfigKeys.NFS_METRICS_PERCENTILES_INTERVALS_KEY);
     return ms.register(new Nfs3Metrics(gatewayName, sessionId, intervals, jm));
   }
   
@@ -217,4 +216,4 @@ public class Nfs3Metrics {
     }
   }
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/570a83ae/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 3725a03..5dae029 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1260,6 +1260,8 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7961. Trigger full block report after hot swapping disk. (Eddy Xu via wang)
 
+    HDFS-7977. NFS couldn't take percentile intervals (brandonli)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/570a83ae/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 9c95287..a49d168 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
@@ -170,6 +170,18 @@ It's strongly recommended for the users to update a few configuration properties
           <value>the_name_of_hdfs_superuser</value>
         </property>
 
+*   Metrics. Like other HDFS daemons, the gateway exposes runtime metrics. It is available at `http://gateway-ip:50079/jmx` as a JSON document.
+    The NFS handler related metrics is exposed under the name "Nfs3Metrics". The latency histograms can be enabled by adding the following
+    property to hdfs-site.xml file.
+
+        <property>
+          <name>nfs.metrics.percentiles.intervals</name>
+          <value>100</value>
+          <description>Enable the latency histograms for read, write and
+             commit requests. The time unit is 100 seconds in this example.
+          </description>
+        </property>
+
 *   JVM and log settings. You can export JVM settings (e.g., heap size and GC log) in
     HADOOP\_NFS3\_OPTS. More NFS related settings can be found in hadoop-env.sh.
     To get NFS debug trace, you can edit the log4j.property file


[03/51] [abbrv] hadoop git commit: MAPREDUCE-6282. Reuse historyFileAbsolute.getFileSystem in CompletedJob#loadFullHistoryData for code optimization. (zxu via rkanter)

Posted by ka...@apache.org.
MAPREDUCE-6282. Reuse historyFileAbsolute.getFileSystem in CompletedJob#loadFullHistoryData for code optimization. (zxu via rkanter)


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

Branch: refs/heads/YARN-2139
Commit: d4f7e2507f4bb024444d172f94e74431bc2f319c
Parents: 75ead27
Author: Robert Kanter <rk...@apache.org>
Authored: Fri Mar 20 13:11:58 2015 -0700
Committer: Robert Kanter <rk...@apache.org>
Committed: Fri Mar 20 13:11:58 2015 -0700

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                             | 4 ++++
 .../java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java     | 4 +---
 2 files changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4f7e250/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 2a4bf0c..48eda8b 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -274,6 +274,10 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-5755. MapTask.MapOutputBuffer#compare/swap should have
     @Override annotation. (ozawa)
 
+    MAPREDUCE-6282. Reuse historyFileAbsolute.getFileSystem in
+    CompletedJob#loadFullHistoryData for code optimization.
+    (zxu via rkanter)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d4f7e250/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
index 1cf63d4..6df8261 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CompletedJob.java
@@ -345,9 +345,7 @@ public class CompletedJob implements org.apache.hadoop.mapreduce.v2.app.job.Job
       JobHistoryParser parser = null;
       try {
         final FileSystem fs = historyFileAbsolute.getFileSystem(conf);
-        parser =
-            new JobHistoryParser(historyFileAbsolute.getFileSystem(conf),
-                historyFileAbsolute);
+        parser = new JobHistoryParser(fs, historyFileAbsolute);
         final Path jobConfPath = new Path(historyFileAbsolute.getParent(),
             JobHistoryUtils.getIntermediateConfFileName(jobId));
         final Configuration conf = new Configuration();


[41/51] [abbrv] hadoop git commit: HDFS-7854. Separate class DataStreamer out of DFSOutputStream. Contributed by Li Bo.

Posted by ka...@apache.org.
HDFS-7854. Separate class DataStreamer out of DFSOutputStream. Contributed by Li Bo.


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

Branch: refs/heads/YARN-2139
Commit: a16bfff71bd7f00e06e1f59bfe5445a154bb8c66
Parents: 570a83a
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Mar 24 11:06:13 2015 -0700
Committer: Jing Zhao <ji...@apache.org>
Committed: Tue Mar 24 11:06:13 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |    3 +
 .../dev-support/findbugsExcludeFile.xml         |    2 +-
 .../org/apache/hadoop/hdfs/DFSOutputStream.java | 1694 ++---------------
 .../org/apache/hadoop/hdfs/DataStreamer.java    | 1754 ++++++++++++++++++
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |    2 +-
 .../apache/hadoop/hdfs/TestDFSOutputStream.java |    5 +-
 .../apache/hadoop/hdfs/TestFileCreation.java    |   18 +-
 7 files changed, 1893 insertions(+), 1585 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a16bfff7/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 5dae029..4ec0891 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -332,6 +332,9 @@ Release 2.8.0 - UNRELEASED
 
     HDFS-7829. Code clean up for LocatedBlock. (Takanobu Asanuma via jing9)
 
+    HDFS-7854. Separate class DataStreamer out of DFSOutputStream. (Li Bo via
+    jing9)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a16bfff7/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index dedeece..224d2fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -77,7 +77,7 @@
       ResponseProccessor is thread that is designed to catch RuntimeException.
      -->
      <Match>
-       <Class name="org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer$ResponseProcessor" />
+       <Class name="org.apache.hadoop.hdfs.DataStreamer$ResponseProcessor" />
        <Method name="run" />
        <Bug pattern="REC_CATCH_EXCEPTION" />
      </Match>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a16bfff7/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 d7d59af..ee3e6f6 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
@@ -17,29 +17,12 @@
  */
 package org.apache.hadoop.hdfs;
 
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
-
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.io.InputStream;
 import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.nio.channels.ClosedChannelException;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.EnumSet;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -52,64 +35,37 @@ import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.Syncable;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
 import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
-import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
 import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.util.ByteArrayManager;
 import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DataChecksum.Type;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Time;
-import org.apache.htrace.NullScope;
 import org.apache.htrace.Sampler;
-import org.apache.htrace.Span;
 import org.apache.htrace.Trace;
-import org.apache.htrace.TraceInfo;
 import org.apache.htrace.TraceScope;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.cache.RemovalListener;
-import com.google.common.cache.RemovalNotification;
 
 
 /****************************************************************
@@ -121,19 +77,11 @@ import com.google.common.cache.RemovalNotification;
  * is typically 512 bytes and has an associated checksum with it.
  *
  * When a client application fills up the currentPacket, it is
- * enqueued into dataQueue.  The DataStreamer thread picks up
- * packets from the dataQueue, sends it to the first datanode in
- * the pipeline and moves it from the dataQueue to the ackQueue.
- * The ResponseProcessor receives acks from the datanodes. When an
- * successful ack for a packet is received from all datanodes, the
- * ResponseProcessor removes the corresponding packet from the
- * ackQueue.
+ * enqueued into the dataQueue of DataStreamer. DataStreamer is a
+ * thread that picks up packets from the dataQueue and sends it to
+ * the first datanode in the pipeline.
  *
- * In case of error, all outstanding packets and moved from
- * ackQueue. A new pipeline is setup by eliminating the bad
- * datanode from the original pipeline. The DataStreamer now
- * starts sending packets from the dataQueue.
-****************************************************************/
+ ****************************************************************/
 @InterfaceAudience.Private
 public class DFSOutputStream extends FSOutputSummer
     implements Syncable, CanSetDropBehind {
@@ -148,45 +96,25 @@ public class DFSOutputStream extends FSOutputSummer
       CryptoProtocolVersion.supported();
 
   private final DFSClient dfsClient;
-  private final long dfsclientSlowLogThresholdMs;
   private final ByteArrayManager byteArrayManager;
-  private Socket s;
   // closed is accessed by different threads under different locks.
   private volatile boolean closed = false;
 
-  private String src;
+  private final String src;
   private final long fileId;
   private final long blockSize;
-  /** Only for DataTransferProtocol.writeBlock(..) */
-  private final DataChecksum checksum4WriteBlock;
-  private final int bytesPerChecksum; 
+  private final int bytesPerChecksum;
 
-  // both dataQueue and ackQueue are protected by dataQueue lock
-  private final LinkedList<DFSPacket> dataQueue = new LinkedList<DFSPacket>();
-  private final LinkedList<DFSPacket> ackQueue = new LinkedList<DFSPacket>();
   private DFSPacket currentPacket = null;
   private DataStreamer streamer;
-  private long currentSeqno = 0;
-  private long lastQueuedSeqno = -1;
-  private long lastAckedSeqno = -1;
-  private long bytesCurBlock = 0; // bytes written in current block
   private int packetSize = 0; // write packet size, not including the header.
   private int chunksPerPacket = 0;
-  private final AtomicReference<IOException> lastException = new AtomicReference<IOException>();
-  private long artificialSlowdown = 0;
   private long lastFlushOffset = 0; // offset when flush was invoked
-  //persist blocks on namenode
-  private final AtomicBoolean persistBlocks = new AtomicBoolean(false);
-  private volatile boolean appendChunk = false;   // appending to existing partial block
   private long initialFileSize = 0; // at time of file open
-  private final Progressable progress;
   private final short blockReplication; // replication factor of file
   private boolean shouldSyncBlock = false; // force blocks to disk upon close
   private final AtomicReference<CachingStrategy> cachingStrategy;
-  private boolean failPacket = false;
   private FileEncryptionInfo fileEncryptionInfo;
-  private static final BlockStoragePolicySuite blockStoragePolicySuite =
-      BlockStoragePolicySuite.createDefaultSuite();
 
   /** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
   private DFSPacket createPacket(int packetSize, int chunksPerPkt, long offsetInBlock,
@@ -207,1326 +135,10 @@ public class DFSOutputStream extends FSOutputSummer
                          getChecksumSize(), lastPacketInBlock);
   }
 
-  /**
-   * For heartbeat packets, create buffer directly by new byte[]
-   * since heartbeats should not be blocked.
-   */
-  private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
-    final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
-    return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO,
-                         getChecksumSize(), false);
-  }
-
-
-  //
-  // The DataStreamer class is responsible for sending data packets to the
-  // datanodes in the pipeline. It retrieves a new blockid and block locations
-  // from the namenode, and starts streaming packets to the pipeline of
-  // Datanodes. Every packet has a sequence number associated with
-  // it. When all the packets for a block are sent out and acks for each
-  // if them are received, the DataStreamer closes the current block.
-  //
-  class DataStreamer extends Daemon {
-    private volatile boolean streamerClosed = false;
-    private ExtendedBlock block; // its length is number of bytes acked
-    private Token<BlockTokenIdentifier> accessToken;
-    private DataOutputStream blockStream;
-    private DataInputStream blockReplyStream;
-    private ResponseProcessor response = null;
-    private volatile DatanodeInfo[] nodes = null; // list of targets for current block
-    private volatile StorageType[] storageTypes = null;
-    private volatile String[] storageIDs = null;
-    private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes =
-        CacheBuilder.newBuilder()
-        .expireAfterWrite(
-            dfsClient.getConf().excludedNodesCacheExpiry,
-            TimeUnit.MILLISECONDS)
-        .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
-          @Override
-          public void onRemoval(
-              RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
-            DFSClient.LOG.info("Removing node " +
-                notification.getKey() + " from the excluded nodes list");
-          }
-        })
-        .build(new CacheLoader<DatanodeInfo, DatanodeInfo>() {
-          @Override
-          public DatanodeInfo load(DatanodeInfo key) throws Exception {
-            return key;
-          }
-        });
-    private String[] favoredNodes;
-    volatile boolean hasError = false;
-    volatile int errorIndex = -1;
-    // Restarting node index
-    AtomicInteger restartingNodeIndex = new AtomicInteger(-1);
-    private long restartDeadline = 0; // Deadline of DN restart
-    private BlockConstructionStage stage;  // block construction stage
-    private long bytesSent = 0; // number of bytes that've been sent
-    private final boolean isLazyPersistFile;
-
-    /** Nodes have been used in the pipeline before and have failed. */
-    private final List<DatanodeInfo> failed = new ArrayList<DatanodeInfo>();
-    /** The last ack sequence number before pipeline failure. */
-    private long lastAckedSeqnoBeforeFailure = -1;
-    private int pipelineRecoveryCount = 0;
-    /** Has the current block been hflushed? */
-    private boolean isHflushed = false;
-    /** Append on an existing block? */
-    private final boolean isAppend;
-
-    private DataStreamer(HdfsFileStatus stat, ExtendedBlock block) {
-      isAppend = false;
-      isLazyPersistFile = isLazyPersist(stat);
-      this.block = block;
-      stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
-    }
-    
-    /**
-     * Construct a data streamer for appending to the last partial block
-     * @param lastBlock last block of the file to be appended
-     * @param stat status of the file to be appended
-     * @param bytesPerChecksum number of bytes per checksum
-     * @throws IOException if error occurs
-     */
-    private DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat,
-        int bytesPerChecksum) throws IOException {
-      isAppend = true;
-      stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
-      block = lastBlock.getBlock();
-      bytesSent = block.getNumBytes();
-      accessToken = lastBlock.getBlockToken();
-      isLazyPersistFile = isLazyPersist(stat);
-      long usedInLastBlock = stat.getLen() % blockSize;
-      int freeInLastBlock = (int)(blockSize - usedInLastBlock);
-
-      // calculate the amount of free space in the pre-existing 
-      // last crc chunk
-      int usedInCksum = (int)(stat.getLen() % bytesPerChecksum);
-      int freeInCksum = bytesPerChecksum - usedInCksum;
-
-      // if there is space in the last block, then we have to 
-      // append to that block
-      if (freeInLastBlock == blockSize) {
-        throw new IOException("The last block for file " + 
-            src + " is full.");
-      }
-
-      if (usedInCksum > 0 && freeInCksum > 0) {
-        // if there is space in the last partial chunk, then 
-        // setup in such a way that the next packet will have only 
-        // one chunk that fills up the partial chunk.
-        //
-        computePacketChunkSize(0, freeInCksum);
-        setChecksumBufSize(freeInCksum);
-        appendChunk = true;
-      } else {
-        // if the remaining space in the block is smaller than 
-        // that expected size of of a packet, then create 
-        // smaller size packet.
-        //
-        computePacketChunkSize(Math.min(dfsClient.getConf().writePacketSize, freeInLastBlock), 
-            bytesPerChecksum);
-      }
-
-      // setup pipeline to append to the last block XXX retries??
-      setPipeline(lastBlock);
-      errorIndex = -1;   // no errors yet.
-      if (nodes.length < 1) {
-        throw new IOException("Unable to retrieve blocks locations " +
-            " for last block " + block +
-            "of file " + src);
-
-      }
-    }
-
-    private void setPipeline(LocatedBlock lb) {
-      setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
-    }
-    private void setPipeline(DatanodeInfo[] nodes, StorageType[] storageTypes,
-        String[] storageIDs) {
-      this.nodes = nodes;
-      this.storageTypes = storageTypes;
-      this.storageIDs = storageIDs;
-    }
-
-    private void setFavoredNodes(String[] favoredNodes) {
-      this.favoredNodes = favoredNodes;
-    }
-
-    /**
-     * Initialize for data streaming
-     */
-    private void initDataStreaming() {
-      this.setName("DataStreamer for file " + src +
-          " block " + block);
-      response = new ResponseProcessor(nodes);
-      response.start();
-      stage = BlockConstructionStage.DATA_STREAMING;
-    }
-    
-    private void endBlock() {
-      if(DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Closing old block " + block);
-      }
-      this.setName("DataStreamer for file " + src);
-      closeResponder();
-      closeStream();
-      setPipeline(null, null, null);
-      stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
-    }
-    
-    /*
-     * streamer thread is the only thread that opens streams to datanode, 
-     * and closes them. Any error recovery is also done by this thread.
-     */
-    @Override
-    public void run() {
-      long lastPacket = Time.monotonicNow();
-      TraceScope scope = NullScope.INSTANCE;
-      while (!streamerClosed && dfsClient.clientRunning) {
-        // if the Responder encountered an error, shutdown Responder
-        if (hasError && response != null) {
-          try {
-            response.close();
-            response.join();
-            response = null;
-          } catch (InterruptedException  e) {
-            DFSClient.LOG.warn("Caught exception ", e);
-          }
-        }
-
-        DFSPacket one;
-        try {
-          // process datanode IO errors if any
-          boolean doSleep = false;
-          if (hasError && (errorIndex >= 0 || restartingNodeIndex.get() >= 0)) {
-            doSleep = processDatanodeError();
-          }
-
-          synchronized (dataQueue) {
-            // wait for a packet to be sent.
-            long now = Time.monotonicNow();
-            while ((!streamerClosed && !hasError && dfsClient.clientRunning 
-                && dataQueue.size() == 0 && 
-                (stage != BlockConstructionStage.DATA_STREAMING || 
-                 stage == BlockConstructionStage.DATA_STREAMING && 
-                 now - lastPacket < dfsClient.getConf().socketTimeout/2)) || doSleep ) {
-              long timeout = dfsClient.getConf().socketTimeout/2 - (now-lastPacket);
-              timeout = timeout <= 0 ? 1000 : timeout;
-              timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
-                 timeout : 1000;
-              try {
-                dataQueue.wait(timeout);
-              } catch (InterruptedException  e) {
-                DFSClient.LOG.warn("Caught exception ", e);
-              }
-              doSleep = false;
-              now = Time.monotonicNow();
-            }
-            if (streamerClosed || hasError || !dfsClient.clientRunning) {
-              continue;
-            }
-            // get packet to be sent.
-            if (dataQueue.isEmpty()) {
-              one = createHeartbeatPacket();
-              assert one != null;
-            } else {
-              one = dataQueue.getFirst(); // regular data packet
-              long parents[] = one.getTraceParents();
-              if (parents.length > 0) {
-                scope = Trace.startSpan("dataStreamer", new TraceInfo(0, parents[0]));
-                // TODO: use setParents API once it's available from HTrace 3.2
-//                scope = Trace.startSpan("dataStreamer", Sampler.ALWAYS);
-//                scope.getSpan().setParents(parents);
-              }
-            }
-          }
-
-          // get new block from namenode.
-          if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
-            if(DFSClient.LOG.isDebugEnabled()) {
-              DFSClient.LOG.debug("Allocating new block");
-            }
-            setPipeline(nextBlockOutputStream());
-            initDataStreaming();
-          } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
-            if(DFSClient.LOG.isDebugEnabled()) {
-              DFSClient.LOG.debug("Append to block " + block);
-            }
-            setupPipelineForAppendOrRecovery();
-            initDataStreaming();
-          }
-
-          long lastByteOffsetInBlock = one.getLastByteOffsetBlock();
-          if (lastByteOffsetInBlock > blockSize) {
-            throw new IOException("BlockSize " + blockSize +
-                " is smaller than data size. " +
-                " Offset of packet in block " + 
-                lastByteOffsetInBlock +
-                " Aborting file " + src);
-          }
-
-          if (one.isLastPacketInBlock()) {
-            // wait for all data packets have been successfully acked
-            synchronized (dataQueue) {
-              while (!streamerClosed && !hasError && 
-                  ackQueue.size() != 0 && dfsClient.clientRunning) {
-                try {
-                  // wait for acks to arrive from datanodes
-                  dataQueue.wait(1000);
-                } catch (InterruptedException  e) {
-                  DFSClient.LOG.warn("Caught exception ", e);
-                }
-              }
-            }
-            if (streamerClosed || hasError || !dfsClient.clientRunning) {
-              continue;
-            }
-            stage = BlockConstructionStage.PIPELINE_CLOSE;
-          }
-          
-          // send the packet
-          Span span = null;
-          synchronized (dataQueue) {
-            // move packet from dataQueue to ackQueue
-            if (!one.isHeartbeatPacket()) {
-              span = scope.detach();
-              one.setTraceSpan(span);
-              dataQueue.removeFirst();
-              ackQueue.addLast(one);
-              dataQueue.notifyAll();
-            }
-          }
-
-          if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug("DataStreamer block " + block +
-                " sending packet " + one);
-          }
-
-          // write out data to remote datanode
-          TraceScope writeScope = Trace.startSpan("writeTo", span);
-          try {
-            one.writeTo(blockStream);
-            blockStream.flush();   
-          } catch (IOException e) {
-            // HDFS-3398 treat primary DN is down since client is unable to 
-            // write to primary DN. If a failed or restarting node has already
-            // been recorded by the responder, the following call will have no 
-            // effect. Pipeline recovery can handle only one node error at a
-            // time. If the primary node fails again during the recovery, it
-            // will be taken out then.
-            tryMarkPrimaryDatanodeFailed();
-            throw e;
-          } finally {
-            writeScope.close();
-          }
-          lastPacket = Time.monotonicNow();
-          
-          // update bytesSent
-          long tmpBytesSent = one.getLastByteOffsetBlock();
-          if (bytesSent < tmpBytesSent) {
-            bytesSent = tmpBytesSent;
-          }
-
-          if (streamerClosed || hasError || !dfsClient.clientRunning) {
-            continue;
-          }
-
-          // Is this block full?
-          if (one.isLastPacketInBlock()) {
-            // wait for the close packet has been acked
-            synchronized (dataQueue) {
-              while (!streamerClosed && !hasError && 
-                  ackQueue.size() != 0 && dfsClient.clientRunning) {
-                dataQueue.wait(1000);// wait for acks to arrive from datanodes
-              }
-            }
-            if (streamerClosed || hasError || !dfsClient.clientRunning) {
-              continue;
-            }
-
-            endBlock();
-          }
-          if (progress != null) { progress.progress(); }
-
-          // This is used by unit test to trigger race conditions.
-          if (artificialSlowdown != 0 && dfsClient.clientRunning) {
-            Thread.sleep(artificialSlowdown); 
-          }
-        } catch (Throwable e) {
-          // Log warning if there was a real error.
-          if (restartingNodeIndex.get() == -1) {
-            // Since their messages are descriptive enough, do not always
-            // log a verbose stack-trace WARN for quota exceptions.
-            if (e instanceof QuotaExceededException) {
-              DFSClient.LOG.debug("DataStreamer Quota Exception", e);
-            } else {
-              DFSClient.LOG.warn("DataStreamer Exception", e);
-            }
-          }
-          if (e instanceof IOException) {
-            setLastException((IOException)e);
-          } else {
-            setLastException(new IOException("DataStreamer Exception: ",e));
-          }
-          hasError = true;
-          if (errorIndex == -1 && restartingNodeIndex.get() == -1) {
-            // Not a datanode issue
-            streamerClosed = true;
-          }
-        } finally {
-          scope.close();
-        }
-      }
-      closeInternal();
-    }
-
-    private void closeInternal() {
-      closeResponder();       // close and join
-      closeStream();
-      streamerClosed = true;
-      setClosed();
-      synchronized (dataQueue) {
-        dataQueue.notifyAll();
-      }
-    }
-
-    /*
-     * close both streamer and DFSOutputStream, should be called only 
-     * by an external thread and only after all data to be sent has 
-     * been flushed to datanode.
-     * 
-     * Interrupt this data streamer if force is true
-     * 
-     * @param force if this data stream is forced to be closed 
-     */
-    void close(boolean force) {
-      streamerClosed = true;
-      synchronized (dataQueue) {
-        dataQueue.notifyAll();
-      }
-      if (force) {
-        this.interrupt();
-      }
-    }
-
-    private void closeResponder() {
-      if (response != null) {
-        try {
-          response.close();
-          response.join();
-        } catch (InterruptedException  e) {
-          DFSClient.LOG.warn("Caught exception ", e);
-        } finally {
-          response = null;
-        }
-      }
-    }
-
-    private void closeStream() {
-      if (blockStream != null) {
-        try {
-          blockStream.close();
-        } catch (IOException e) {
-          setLastException(e);
-        } finally {
-          blockStream = null;
-        }
-      }
-      if (blockReplyStream != null) {
-        try {
-          blockReplyStream.close();
-        } catch (IOException e) {
-          setLastException(e);
-        } finally {
-          blockReplyStream = null;
-        }
-      }
-      if (null != s) {
-        try {
-          s.close();
-        } catch (IOException e) {
-          setLastException(e);
-        } finally {
-          s = null;
-        }
-      }
-    }
-
-    // The following synchronized methods are used whenever 
-    // errorIndex or restartingNodeIndex is set. This is because
-    // check & set needs to be atomic. Simply reading variables
-    // does not require a synchronization. When responder is
-    // not running (e.g. during pipeline recovery), there is no
-    // need to use these methods.
-
-    /** Set the error node index. Called by responder */
-    synchronized void setErrorIndex(int idx) {
-      errorIndex = idx;
-    }
-
-    /** Set the restarting node index. Called by responder */
-    synchronized void setRestartingNodeIndex(int idx) {
-      restartingNodeIndex.set(idx);
-      // If the data streamer has already set the primary node
-      // bad, clear it. It is likely that the write failed due to
-      // the DN shutdown. Even if it was a real failure, the pipeline
-      // recovery will take care of it.
-      errorIndex = -1;      
-    }
-
-    /**
-     * This method is used when no explicit error report was received,
-     * but something failed. When the primary node is a suspect or
-     * unsure about the cause, the primary node is marked as failed.
-     */
-    synchronized void tryMarkPrimaryDatanodeFailed() {
-      // There should be no existing error and no ongoing restart.
-      if ((errorIndex == -1) && (restartingNodeIndex.get() == -1)) {
-        errorIndex = 0;
-      }
-    }
-
-    /**
-     * Examine whether it is worth waiting for a node to restart.
-     * @param index the node index
-     */
-    boolean shouldWaitForRestart(int index) {
-      // Only one node in the pipeline.
-      if (nodes.length == 1) {
-        return true;
-      }
-
-      // Is it a local node?
-      InetAddress addr = null;
-      try {
-        addr = InetAddress.getByName(nodes[index].getIpAddr());
-      } catch (java.net.UnknownHostException e) {
-        // we are passing an ip address. this should not happen.
-        assert false;
-      }
-
-      if (addr != null && NetUtils.isLocalAddress(addr)) {
-        return true;
-      }
-      return false;
-    }
-
-    //
-    // Processes responses from the datanodes.  A packet is removed
-    // from the ackQueue when its response arrives.
-    //
-    private class ResponseProcessor extends Daemon {
-
-      private volatile boolean responderClosed = false;
-      private DatanodeInfo[] targets = null;
-      private boolean isLastPacketInBlock = false;
-
-      ResponseProcessor (DatanodeInfo[] targets) {
-        this.targets = targets;
-      }
-
-      @Override
-      public void run() {
-
-        setName("ResponseProcessor for block " + block);
-        PipelineAck ack = new PipelineAck();
-
-        TraceScope scope = NullScope.INSTANCE;
-        while (!responderClosed && dfsClient.clientRunning && !isLastPacketInBlock) {
-          // process responses from datanodes.
-          try {
-            // read an ack from the pipeline
-            long begin = Time.monotonicNow();
-            ack.readFields(blockReplyStream);
-            long duration = Time.monotonicNow() - begin;
-            if (duration > dfsclientSlowLogThresholdMs
-                && ack.getSeqno() != DFSPacket.HEART_BEAT_SEQNO) {
-              DFSClient.LOG
-                  .warn("Slow ReadProcessor read fields took " + duration
-                      + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
-                      + ack + ", targets: " + Arrays.asList(targets));
-            } else if (DFSClient.LOG.isDebugEnabled()) {
-              DFSClient.LOG.debug("DFSClient " + ack);
-            }
-
-            long seqno = ack.getSeqno();
-            // processes response status from datanodes.
-            for (int i = ack.getNumOfReplies()-1; i >=0  && dfsClient.clientRunning; i--) {
-              final Status reply = PipelineAck.getStatusFromHeader(ack
-                .getReply(i));
-              // Restart will not be treated differently unless it is
-              // the local node or the only one in the pipeline.
-              if (PipelineAck.isRestartOOBStatus(reply) &&
-                  shouldWaitForRestart(i)) {
-                restartDeadline = dfsClient.getConf().datanodeRestartTimeout
-                    + Time.monotonicNow();
-                setRestartingNodeIndex(i);
-                String message = "A datanode is restarting: " + targets[i];
-                DFSClient.LOG.info(message);
-               throw new IOException(message);
-              }
-              // node error
-              if (reply != SUCCESS) {
-                setErrorIndex(i); // first bad datanode
-                throw new IOException("Bad response " + reply +
-                    " for block " + block +
-                    " from datanode " + 
-                    targets[i]);
-              }
-            }
-            
-            assert seqno != PipelineAck.UNKOWN_SEQNO : 
-              "Ack for unknown seqno should be a failed ack: " + ack;
-            if (seqno == DFSPacket.HEART_BEAT_SEQNO) {  // a heartbeat ack
-              continue;
-            }
-
-            // a success ack for a data packet
-            DFSPacket one;
-            synchronized (dataQueue) {
-              one = ackQueue.getFirst();
-            }
-            if (one.getSeqno() != seqno) {
-              throw new IOException("ResponseProcessor: Expecting seqno " +
-                                    " for block " + block +
-                                    one.getSeqno() + " but received " + seqno);
-            }
-            isLastPacketInBlock = one.isLastPacketInBlock();
-
-            // Fail the packet write for testing in order to force a
-            // pipeline recovery.
-            if (DFSClientFaultInjector.get().failPacket() &&
-                isLastPacketInBlock) {
-              failPacket = true;
-              throw new IOException(
-                    "Failing the last packet for testing.");
-            }
-              
-            // update bytesAcked
-            block.setNumBytes(one.getLastByteOffsetBlock());
-
-            synchronized (dataQueue) {
-              scope = Trace.continueSpan(one.getTraceSpan());
-              one.setTraceSpan(null);
-              lastAckedSeqno = seqno;
-              ackQueue.removeFirst();
-              dataQueue.notifyAll();
-
-              one.releaseBuffer(byteArrayManager);
-            }
-          } catch (Exception e) {
-            if (!responderClosed) {
-              if (e instanceof IOException) {
-                setLastException((IOException)e);
-              }
-              hasError = true;
-              // If no explicit error report was received, mark the primary
-              // node as failed.
-              tryMarkPrimaryDatanodeFailed();
-              synchronized (dataQueue) {
-                dataQueue.notifyAll();
-              }
-              if (restartingNodeIndex.get() == -1) {
-                DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception "
-                     + " for block " + block, e);
-              }
-              responderClosed = true;
-            }
-          } finally {
-            scope.close();
-          }
-        }
-      }
-
-      void close() {
-        responderClosed = true;
-        this.interrupt();
-      }
-    }
-
-    // If this stream has encountered any errors so far, shutdown 
-    // threads and mark stream as closed. Returns true if we should
-    // sleep for a while after returning from this call.
-    //
-    private boolean processDatanodeError() throws IOException {
-      if (response != null) {
-        DFSClient.LOG.info("Error Recovery for " + block +
-        " waiting for responder to exit. ");
-        return true;
-      }
-      closeStream();
-
-      // move packets from ack queue to front of the data queue
-      synchronized (dataQueue) {
-        dataQueue.addAll(0, ackQueue);
-        ackQueue.clear();
-      }
-
-      // Record the new pipeline failure recovery.
-      if (lastAckedSeqnoBeforeFailure != lastAckedSeqno) {
-         lastAckedSeqnoBeforeFailure = lastAckedSeqno;
-         pipelineRecoveryCount = 1;
-      } else {
-        // If we had to recover the pipeline five times in a row for the
-        // same packet, this client likely has corrupt data or corrupting
-        // during transmission.
-        if (++pipelineRecoveryCount > 5) {
-          DFSClient.LOG.warn("Error recovering pipeline for writing " +
-              block + ". Already retried 5 times for the same packet.");
-          lastException.set(new IOException("Failing write. Tried pipeline " +
-              "recovery 5 times without success."));
-          streamerClosed = true;
-          return false;
-        }
-      }
-      boolean doSleep = setupPipelineForAppendOrRecovery();
-      
-      if (!streamerClosed && dfsClient.clientRunning) {
-        if (stage == BlockConstructionStage.PIPELINE_CLOSE) {
-
-          // If we had an error while closing the pipeline, we go through a fast-path
-          // where the BlockReceiver does not run. Instead, the DataNode just finalizes
-          // the block immediately during the 'connect ack' process. So, we want to pull
-          // the end-of-block packet from the dataQueue, since we don't actually have
-          // a true pipeline to send it over.
-          //
-          // We also need to set lastAckedSeqno to the end-of-block Packet's seqno, so that
-          // a client waiting on close() will be aware that the flush finished.
-          synchronized (dataQueue) {
-            DFSPacket endOfBlockPacket = dataQueue.remove();  // remove the end of block packet
-            Span span = endOfBlockPacket.getTraceSpan();
-            if (span != null) {
-              // Close any trace span associated with this Packet
-              TraceScope scope = Trace.continueSpan(span);
-              scope.close();
-            }
-            assert endOfBlockPacket.isLastPacketInBlock();
-            assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1;
-            lastAckedSeqno = endOfBlockPacket.getSeqno();
-            dataQueue.notifyAll();
-          }
-          endBlock();
-        } else {
-          initDataStreaming();
-        }
-      }
-      
-      return doSleep;
-    }
-
-    private void setHflush() {
-      isHflushed = true;
-    }
-
-    private int findNewDatanode(final DatanodeInfo[] original
-        ) throws IOException {
-      if (nodes.length != original.length + 1) {
-        throw new IOException(
-            new StringBuilder()
-            .append("Failed to replace a bad datanode on the existing pipeline ")
-            .append("due to no more good datanodes being available to try. ")
-            .append("(Nodes: current=").append(Arrays.asList(nodes))
-            .append(", original=").append(Arrays.asList(original)).append("). ")
-            .append("The current failed datanode replacement policy is ")
-            .append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
-            .append("a client may configure this via '")
-            .append(DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY)
-            .append("' in its configuration.")
-            .toString());
-      }
-      for(int i = 0; i < nodes.length; i++) {
-        int j = 0;
-        for(; j < original.length && !nodes[i].equals(original[j]); j++);
-        if (j == original.length) {
-          return i;
-        }
-      }
-      throw new IOException("Failed: new datanode not found: nodes="
-          + Arrays.asList(nodes) + ", original=" + Arrays.asList(original));
-    }
-
-    private void addDatanode2ExistingPipeline() throws IOException {
-      if (DataTransferProtocol.LOG.isDebugEnabled()) {
-        DataTransferProtocol.LOG.debug("lastAckedSeqno = " + lastAckedSeqno);
-      }
-      /*
-       * Is data transfer necessary?  We have the following cases.
-       * 
-       * Case 1: Failure in Pipeline Setup
-       * - Append
-       *    + Transfer the stored replica, which may be a RBW or a finalized.
-       * - Create
-       *    + If no data, then no transfer is required.
-       *    + If there are data written, transfer RBW. This case may happens 
-       *      when there are streaming failure earlier in this pipeline.
-       *
-       * Case 2: Failure in Streaming
-       * - Append/Create:
-       *    + transfer RBW
-       * 
-       * Case 3: Failure in Close
-       * - Append/Create:
-       *    + no transfer, let NameNode replicates the block.
-       */
-      if (!isAppend && lastAckedSeqno < 0
-          && stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
-        //no data have been written
-        return;
-      } else if (stage == BlockConstructionStage.PIPELINE_CLOSE
-          || stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
-        //pipeline is closing
-        return;
-      }
-
-      //get a new datanode
-      final DatanodeInfo[] original = nodes;
-      final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode(
-          src, fileId, block, nodes, storageIDs,
-          failed.toArray(new DatanodeInfo[failed.size()]),
-          1, dfsClient.clientName);
-      setPipeline(lb);
-
-      //find the new datanode
-      final int d = findNewDatanode(original);
-
-      //transfer replica
-      final DatanodeInfo src = d == 0? nodes[1]: nodes[d - 1];
-      final DatanodeInfo[] targets = {nodes[d]};
-      final StorageType[] targetStorageTypes = {storageTypes[d]};
-      transfer(src, targets, targetStorageTypes, lb.getBlockToken());
-    }
-
-    private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
-        final StorageType[] targetStorageTypes,
-        final Token<BlockTokenIdentifier> blockToken) throws IOException {
-      //transfer replica to the new datanode
-      Socket sock = null;
-      DataOutputStream out = null;
-      DataInputStream in = null;
-      try {
-        sock = createSocketForPipeline(src, 2, dfsClient);
-        final long writeTimeout = dfsClient.getDatanodeWriteTimeout(2);
-        
-        OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
-        InputStream unbufIn = NetUtils.getInputStream(sock);
-        IOStreamPair saslStreams = dfsClient.saslClient.socketSend(sock,
-          unbufOut, unbufIn, dfsClient, blockToken, src);
-        unbufOut = saslStreams.out;
-        unbufIn = saslStreams.in;
-        out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-            HdfsConstants.SMALL_BUFFER_SIZE));
-        in = new DataInputStream(unbufIn);
-
-        //send the TRANSFER_BLOCK request
-        new Sender(out).transferBlock(block, blockToken, dfsClient.clientName,
-            targets, targetStorageTypes);
-        out.flush();
-
-        //ack
-        BlockOpResponseProto response =
-          BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
-        if (SUCCESS != response.getStatus()) {
-          throw new IOException("Failed to add a datanode");
-        }
-      } finally {
-        IOUtils.closeStream(in);
-        IOUtils.closeStream(out);
-        IOUtils.closeSocket(sock);
-      }
-    }
-
-    /**
-     * Open a DataOutputStream to a DataNode pipeline so that 
-     * it can be written to.
-     * This happens when a file is appended or data streaming fails
-     * It keeps on trying until a pipeline is setup
-     */
-    private boolean setupPipelineForAppendOrRecovery() throws IOException {
-      // check number of datanodes
-      if (nodes == null || nodes.length == 0) {
-        String msg = "Could not get block locations. " + "Source file \""
-            + src + "\" - Aborting...";
-        DFSClient.LOG.warn(msg);
-        setLastException(new IOException(msg));
-        streamerClosed = true;
-        return false;
-      }
-      
-      boolean success = false;
-      long newGS = 0L;
-      while (!success && !streamerClosed && dfsClient.clientRunning) {
-        // Sleep before reconnect if a dn is restarting.
-        // This process will be repeated until the deadline or the datanode
-        // starts back up.
-        if (restartingNodeIndex.get() >= 0) {
-          // 4 seconds or the configured deadline period, whichever is shorter.
-          // This is the retry interval and recovery will be retried in this
-          // interval until timeout or success.
-          long delay = Math.min(dfsClient.getConf().datanodeRestartTimeout,
-              4000L);
-          try {
-            Thread.sleep(delay);
-          } catch (InterruptedException ie) {
-            lastException.set(new IOException("Interrupted while waiting for " +
-                "datanode to restart. " + nodes[restartingNodeIndex.get()]));
-            streamerClosed = true;
-            return false;
-          }
-        }
-        boolean isRecovery = hasError;
-        // remove bad datanode from list of datanodes.
-        // If errorIndex was not set (i.e. appends), then do not remove 
-        // any datanodes
-        // 
-        if (errorIndex >= 0) {
-          StringBuilder pipelineMsg = new StringBuilder();
-          for (int j = 0; j < nodes.length; j++) {
-            pipelineMsg.append(nodes[j]);
-            if (j < nodes.length - 1) {
-              pipelineMsg.append(", ");
-            }
-          }
-          if (nodes.length <= 1) {
-            lastException.set(new IOException("All datanodes " + pipelineMsg
-                + " are bad. Aborting..."));
-            streamerClosed = true;
-            return false;
-          }
-          DFSClient.LOG.warn("Error Recovery for block " + block +
-              " in pipeline " + pipelineMsg + 
-              ": bad datanode " + nodes[errorIndex]);
-          failed.add(nodes[errorIndex]);
-
-          DatanodeInfo[] newnodes = new DatanodeInfo[nodes.length-1];
-          arraycopy(nodes, newnodes, errorIndex);
-
-          final StorageType[] newStorageTypes = new StorageType[newnodes.length];
-          arraycopy(storageTypes, newStorageTypes, errorIndex);
-
-          final String[] newStorageIDs = new String[newnodes.length];
-          arraycopy(storageIDs, newStorageIDs, errorIndex);
-          
-          setPipeline(newnodes, newStorageTypes, newStorageIDs);
-
-          // Just took care of a node error while waiting for a node restart
-          if (restartingNodeIndex.get() >= 0) {
-            // If the error came from a node further away than the restarting
-            // node, the restart must have been complete.
-            if (errorIndex > restartingNodeIndex.get()) {
-              restartingNodeIndex.set(-1);
-            } else if (errorIndex < restartingNodeIndex.get()) {
-              // the node index has shifted.
-              restartingNodeIndex.decrementAndGet();
-            } else {
-              // this shouldn't happen...
-              assert false;
-            }
-          }
-
-          if (restartingNodeIndex.get() == -1) {
-            hasError = false;
-          }
-          lastException.set(null);
-          errorIndex = -1;
-        }
-
-        // Check if replace-datanode policy is satisfied.
-        if (dfsClient.dtpReplaceDatanodeOnFailure.satisfy(blockReplication,
-            nodes, isAppend, isHflushed)) {
-          try {
-            addDatanode2ExistingPipeline();
-          } catch(IOException ioe) {
-            if (!dfsClient.dtpReplaceDatanodeOnFailure.isBestEffort()) {
-              throw ioe;
-            }
-            DFSClient.LOG.warn("Failed to replace datanode."
-                + " Continue with the remaining datanodes since "
-                + DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_BEST_EFFORT_KEY
-                + " is set to true.", ioe);
-          }
-        }
-
-        // get a new generation stamp and an access token
-        LocatedBlock lb = dfsClient.namenode.updateBlockForPipeline(block, dfsClient.clientName);
-        newGS = lb.getBlock().getGenerationStamp();
-        accessToken = lb.getBlockToken();
-        
-        // set up the pipeline again with the remaining nodes
-        if (failPacket) { // for testing
-          success = createBlockOutputStream(nodes, storageTypes, newGS, isRecovery);
-          failPacket = false;
-          try {
-            // Give DNs time to send in bad reports. In real situations,
-            // good reports should follow bad ones, if client committed
-            // with those nodes.
-            Thread.sleep(2000);
-          } catch (InterruptedException ie) {}
-        } else {
-          success = createBlockOutputStream(nodes, storageTypes, newGS, isRecovery);
-        }
-
-        if (restartingNodeIndex.get() >= 0) {
-          assert hasError == true;
-          // check errorIndex set above
-          if (errorIndex == restartingNodeIndex.get()) {
-            // ignore, if came from the restarting node
-            errorIndex = -1;
-          }
-          // still within the deadline
-          if (Time.monotonicNow() < restartDeadline) {
-            continue; // with in the deadline
-          }
-          // expired. declare the restarting node dead
-          restartDeadline = 0;
-          int expiredNodeIndex = restartingNodeIndex.get();
-          restartingNodeIndex.set(-1);
-          DFSClient.LOG.warn("Datanode did not restart in time: " +
-              nodes[expiredNodeIndex]);
-          // Mark the restarting node as failed. If there is any other failed
-          // node during the last pipeline construction attempt, it will not be
-          // overwritten/dropped. In this case, the restarting node will get
-          // excluded in the following attempt, if it still does not come up.
-          if (errorIndex == -1) {
-            errorIndex = expiredNodeIndex;
-          }
-          // From this point on, normal pipeline recovery applies.
-        }
-      } // while
-
-      if (success) {
-        // update pipeline at the namenode
-        ExtendedBlock newBlock = new ExtendedBlock(
-            block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
-        dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
-            nodes, storageIDs);
-        // update client side generation stamp
-        block = newBlock;
-      }
-      return false; // do not sleep, continue processing
-    }
-
-    /**
-     * Open a DataOutputStream to a DataNode so that it can be written to.
-     * This happens when a file is created and each time a new block is allocated.
-     * Must get block ID and the IDs of the destinations from the namenode.
-     * Returns the list of target datanodes.
-     */
-    private LocatedBlock nextBlockOutputStream() throws IOException {
-      LocatedBlock lb = null;
-      DatanodeInfo[] nodes = null;
-      StorageType[] storageTypes = null;
-      int count = dfsClient.getConf().nBlockWriteRetry;
-      boolean success = false;
-      ExtendedBlock oldBlock = block;
-      do {
-        hasError = false;
-        lastException.set(null);
-        errorIndex = -1;
-        success = false;
-
-        DatanodeInfo[] excluded =
-            excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
-            .keySet()
-            .toArray(new DatanodeInfo[0]);
-        block = oldBlock;
-        lb = locateFollowingBlock(excluded.length > 0 ? excluded : null);
-        block = lb.getBlock();
-        block.setNumBytes(0);
-        bytesSent = 0;
-        accessToken = lb.getBlockToken();
-        nodes = lb.getLocations();
-        storageTypes = lb.getStorageTypes();
-
-        //
-        // Connect to first DataNode in the list.
-        //
-        success = createBlockOutputStream(nodes, storageTypes, 0L, false);
-
-        if (!success) {
-          DFSClient.LOG.info("Abandoning " + block);
-          dfsClient.namenode.abandonBlock(block, fileId, src,
-              dfsClient.clientName);
-          block = null;
-          DFSClient.LOG.info("Excluding datanode " + nodes[errorIndex]);
-          excludedNodes.put(nodes[errorIndex], nodes[errorIndex]);
-        }
-      } while (!success && --count >= 0);
-
-      if (!success) {
-        throw new IOException("Unable to create new block.");
-      }
-      return lb;
-    }
-
-    // connects to the first datanode in the pipeline
-    // Returns true if success, otherwise return failure.
-    //
-    private boolean createBlockOutputStream(DatanodeInfo[] nodes,
-        StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
-      if (nodes.length == 0) {
-        DFSClient.LOG.info("nodes are empty for write pipeline of block "
-            + block);
-        return false;
-      }
-      Status pipelineStatus = SUCCESS;
-      String firstBadLink = "";
-      boolean checkRestart = false;
-      if (DFSClient.LOG.isDebugEnabled()) {
-        for (int i = 0; i < nodes.length; i++) {
-          DFSClient.LOG.debug("pipeline = " + nodes[i]);
-        }
-      }
-
-      // persist blocks on namenode on next flush
-      persistBlocks.set(true);
-
-      int refetchEncryptionKey = 1;
-      while (true) {
-        boolean result = false;
-        DataOutputStream out = null;
-        try {
-          assert null == s : "Previous socket unclosed";
-          assert null == blockReplyStream : "Previous blockReplyStream unclosed";
-          s = createSocketForPipeline(nodes[0], nodes.length, dfsClient);
-          long writeTimeout = dfsClient.getDatanodeWriteTimeout(nodes.length);
-          
-          OutputStream unbufOut = NetUtils.getOutputStream(s, writeTimeout);
-          InputStream unbufIn = NetUtils.getInputStream(s);
-          IOStreamPair saslStreams = dfsClient.saslClient.socketSend(s,
-            unbufOut, unbufIn, dfsClient, accessToken, nodes[0]);
-          unbufOut = saslStreams.out;
-          unbufIn = saslStreams.in;
-          out = new DataOutputStream(new BufferedOutputStream(unbufOut,
-              HdfsConstants.SMALL_BUFFER_SIZE));
-          blockReplyStream = new DataInputStream(unbufIn);
-  
-          //
-          // Xmit header info to datanode
-          //
-  
-          BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
-
-          // We cannot change the block length in 'block' as it counts the number
-          // of bytes ack'ed.
-          ExtendedBlock blockCopy = new ExtendedBlock(block);
-          blockCopy.setNumBytes(blockSize);
-
-          boolean[] targetPinnings = getPinnings(nodes, true);
-          // send the request
-          new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
-              dfsClient.clientName, nodes, nodeStorageTypes, null, bcs, 
-              nodes.length, block.getNumBytes(), bytesSent, newGS,
-              checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
-            (targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
-  
-          // receive ack for connect
-          BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
-              PBHelper.vintPrefixed(blockReplyStream));
-          pipelineStatus = resp.getStatus();
-          firstBadLink = resp.getFirstBadLink();
-          
-          // Got an restart OOB ack.
-          // If a node is already restarting, this status is not likely from
-          // the same node. If it is from a different node, it is not
-          // from the local datanode. Thus it is safe to treat this as a
-          // regular node error.
-          if (PipelineAck.isRestartOOBStatus(pipelineStatus) &&
-            restartingNodeIndex.get() == -1) {
-            checkRestart = true;
-            throw new IOException("A datanode is restarting.");
-          }
-
-          String logInfo = "ack with firstBadLink as " + firstBadLink;
-          DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
-
-          assert null == blockStream : "Previous blockStream unclosed";
-          blockStream = out;
-          result =  true; // success
-          restartingNodeIndex.set(-1);
-          hasError = false;
-        } catch (IOException ie) {
-          if (restartingNodeIndex.get() == -1) {
-            DFSClient.LOG.info("Exception in createBlockOutputStream", ie);
-          }
-          if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
-            DFSClient.LOG.info("Will fetch a new encryption key and retry, " 
-                + "encryption key was invalid when connecting to "
-                + nodes[0] + " : " + ie);
-            // The encryption key used is invalid.
-            refetchEncryptionKey--;
-            dfsClient.clearDataEncryptionKey();
-            // Don't close the socket/exclude this node just yet. Try again with
-            // a new encryption key.
-            continue;
-          }
-  
-          // find the datanode that matches
-          if (firstBadLink.length() != 0) {
-            for (int i = 0; i < nodes.length; i++) {
-              // NB: Unconditionally using the xfer addr w/o hostname
-              if (firstBadLink.equals(nodes[i].getXferAddr())) {
-                errorIndex = i;
-                break;
-              }
-            }
-          } else {
-            assert checkRestart == false;
-            errorIndex = 0;
-          }
-          // Check whether there is a restart worth waiting for.
-          if (checkRestart && shouldWaitForRestart(errorIndex)) {
-            restartDeadline = dfsClient.getConf().datanodeRestartTimeout +
-                Time.monotonicNow();
-            restartingNodeIndex.set(errorIndex);
-            errorIndex = -1;
-            DFSClient.LOG.info("Waiting for the datanode to be restarted: " +
-                nodes[restartingNodeIndex.get()]);
-          }
-          hasError = true;
-          setLastException(ie);
-          result =  false;  // error
-        } finally {
-          if (!result) {
-            IOUtils.closeSocket(s);
-            s = null;
-            IOUtils.closeStream(out);
-            out = null;
-            IOUtils.closeStream(blockReplyStream);
-            blockReplyStream = null;
-          }
-        }
-        return result;
-      }
-    }
-
-    private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
-      if (favoredNodes == null) {
-        return null;
-      } else {
-        boolean[] pinnings = new boolean[nodes.length];
-        HashSet<String> favoredSet =
-            new HashSet<String>(Arrays.asList(favoredNodes));
-        for (int i = 0; i < nodes.length; i++) {
-          pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
-          if (DFSClient.LOG.isDebugEnabled()) {
-            DFSClient.LOG.debug(nodes[i].getXferAddrWithHostname() +
-                " was chosen by name node (favored=" + pinnings[i] +
-                ").");
-          }
-        }
-        if (shouldLog && !favoredSet.isEmpty()) {
-          // There is one or more favored nodes that were not allocated.
-          DFSClient.LOG.warn(
-              "These favored nodes were specified but not chosen: " +
-              favoredSet +
-              " Specified favored nodes: " + Arrays.toString(favoredNodes));
-
-        }
-        return pinnings;
-      }
-    }
-
-    private LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)  throws IOException {
-      int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
-      long sleeptime = dfsClient.getConf().
-          blockWriteLocateFollowingInitialDelayMs;
-      while (true) {
-        long localstart = Time.monotonicNow();
-        while (true) {
-          try {
-            return dfsClient.namenode.addBlock(src, dfsClient.clientName,
-                block, excludedNodes, fileId, favoredNodes);
-          } catch (RemoteException e) {
-            IOException ue = 
-              e.unwrapRemoteException(FileNotFoundException.class,
-                                      AccessControlException.class,
-                                      NSQuotaExceededException.class,
-                                      DSQuotaExceededException.class,
-                                      UnresolvedPathException.class);
-            if (ue != e) { 
-              throw ue; // no need to retry these exceptions
-            }
-            
-            
-            if (NotReplicatedYetException.class.getName().
-                equals(e.getClassName())) {
-              if (retries == 0) { 
-                throw e;
-              } else {
-                --retries;
-                DFSClient.LOG.info("Exception while adding a block", e);
-                long elapsed = Time.monotonicNow() - localstart;
-                if (elapsed > 5000) {
-                  DFSClient.LOG.info("Waiting for replication for "
-                      + (elapsed / 1000) + " seconds");
-                }
-                try {
-                  DFSClient.LOG.warn("NotReplicatedYetException sleeping " + src
-                      + " retries left " + retries);
-                  Thread.sleep(sleeptime);
-                  sleeptime *= 2;
-                } catch (InterruptedException ie) {
-                  DFSClient.LOG.warn("Caught exception ", ie);
-                }
-              }
-            } else {
-              throw e;
-            }
-
-          }
-        }
-      } 
-    }
-
-    ExtendedBlock getBlock() {
-      return block;
-    }
-
-    DatanodeInfo[] getNodes() {
-      return nodes;
-    }
-
-    Token<BlockTokenIdentifier> getBlockToken() {
-      return accessToken;
-    }
-
-    private void setLastException(IOException e) {
-      lastException.compareAndSet(null, e);
-    }
-  }
-
-  /**
-   * Create a socket for a write pipeline
-   * @param first the first datanode 
-   * @param length the pipeline length
-   * @param client client
-   * @return the socket connected to the first datanode
-   */
-  static Socket createSocketForPipeline(final DatanodeInfo first,
-      final int length, final DFSClient client) throws IOException {
-    final String dnAddr = first.getXferAddr(
-        client.getConf().connectToDnViaHostname);
-    if (DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
-    }
-    final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
-    final Socket sock = client.socketFactory.createSocket();
-    final int timeout = client.getDatanodeReadTimeout(length);
-    NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), client.getConf().socketTimeout);
-    sock.setSoTimeout(timeout);
-    sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
-    if(DFSClient.LOG.isDebugEnabled()) {
-      DFSClient.LOG.debug("Send buf size " + sock.getSendBufferSize());
-    }
-    return sock;
-  }
-
   @Override
   protected void checkClosed() throws IOException {
     if (isClosed()) {
-      IOException e = lastException.get();
+      IOException e = streamer.getLastException().get();
       throw e != null ? e : new ClosedChannelException();
     }
   }
@@ -1536,7 +148,7 @@ public class DFSOutputStream extends FSOutputSummer
   //
   @VisibleForTesting
   public synchronized DatanodeInfo[] getPipeline() {
-    if (streamer == null) {
+    if (streamer.streamerClosed()) {
       return null;
     }
     DatanodeInfo[] currentNodes = streamer.getNodes();
@@ -1556,7 +168,7 @@ public class DFSOutputStream extends FSOutputSummer
    */
   private static DataChecksum getChecksum4Compute(DataChecksum checksum,
       HdfsFileStatus stat) {
-    if (isLazyPersist(stat) && stat.getReplication() == 1) {
+    if (DataStreamer.isLazyPersist(stat) && stat.getReplication() == 1) {
       // do not compute checksum for writing to single replica to memory
       return DataChecksum.newDataChecksum(Type.NULL,
           checksum.getBytesPerChecksum());
@@ -1573,7 +185,6 @@ public class DFSOutputStream extends FSOutputSummer
     this.blockSize = stat.getBlockSize();
     this.blockReplication = stat.getReplication();
     this.fileEncryptionInfo = stat.getFileEncryptionInfo();
-    this.progress = progress;
     this.cachingStrategy = new AtomicReference<CachingStrategy>(
         dfsClient.getDefaultWriteCachingStrategy());
     if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
@@ -1591,10 +202,6 @@ public class DFSOutputStream extends FSOutputSummer
           + DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
           + ") must divide block size (=" + blockSize + ").");
     }
-    this.checksum4WriteBlock = checksum;
-
-    this.dfsclientSlowLogThresholdMs =
-      dfsClient.getConf().dfsclientSlowIoWarningThresholdMs;
     this.byteArrayManager = dfsClient.getClientContext().getByteArrayManager();
   }
 
@@ -1607,7 +214,8 @@ public class DFSOutputStream extends FSOutputSummer
 
     computePacketChunkSize(dfsClient.getConf().writePacketSize, bytesPerChecksum);
 
-    streamer = new DataStreamer(stat, null);
+    streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
+        cachingStrategy, byteArrayManager);
     if (favoredNodes != null && favoredNodes.length != 0) {
       streamer.setFavoredNodes(favoredNodes);
     }
@@ -1676,18 +284,57 @@ public class DFSOutputStream extends FSOutputSummer
     this(dfsClient, src, progress, stat, checksum);
     initialFileSize = stat.getLen(); // length of file when opened
 
+    this.fileEncryptionInfo = stat.getFileEncryptionInfo();
+
     // The last partial block of the file has to be filled.
     if (!toNewBlock && lastBlock != null) {
       // indicate that we are appending to an existing block
-      bytesCurBlock = lastBlock.getBlockSize();
-      streamer = new DataStreamer(lastBlock, stat, bytesPerChecksum);
+      streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress, checksum,
+          cachingStrategy, byteArrayManager);
+      streamer.setBytesCurBlock(lastBlock.getBlockSize());
+      adjustPacketChunkSize(stat);
+      streamer.setPipelineInConstruction(lastBlock);
     } else {
       computePacketChunkSize(dfsClient.getConf().writePacketSize,
           bytesPerChecksum);
-      streamer = new DataStreamer(stat,
-          lastBlock != null ? lastBlock.getBlock() : null);
+      streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
+          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager);
+    }
+  }
+
+  private void adjustPacketChunkSize(HdfsFileStatus stat) throws IOException{
+
+    long usedInLastBlock = stat.getLen() % blockSize;
+    int freeInLastBlock = (int)(blockSize - usedInLastBlock);
+
+    // calculate the amount of free space in the pre-existing
+    // last crc chunk
+    int usedInCksum = (int)(stat.getLen() % bytesPerChecksum);
+    int freeInCksum = bytesPerChecksum - usedInCksum;
+
+    // if there is space in the last block, then we have to
+    // append to that block
+    if (freeInLastBlock == blockSize) {
+      throw new IOException("The last block for file " +
+          src + " is full.");
+    }
+
+    if (usedInCksum > 0 && freeInCksum > 0) {
+      // if there is space in the last partial chunk, then
+      // setup in such a way that the next packet will have only
+      // one chunk that fills up the partial chunk.
+      //
+      computePacketChunkSize(0, freeInCksum);
+      setChecksumBufSize(freeInCksum);
+      streamer.setAppendChunk(true);
+    } else {
+      // if the remaining space in the block is smaller than
+      // that expected size of of a packet, then create
+      // smaller size packet.
+      //
+      computePacketChunkSize(Math.min(dfsClient.getConf().writePacketSize, freeInLastBlock),
+          bytesPerChecksum);
     }
-    this.fileEncryptionInfo = stat.getFileEncryptionInfo();
   }
 
   static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
@@ -1708,12 +355,6 @@ public class DFSOutputStream extends FSOutputSummer
       scope.close();
     }
   }
-  
-  private static boolean isLazyPersist(HdfsFileStatus stat) {
-    final BlockStoragePolicy p = blockStoragePolicySuite.getPolicy(
-        HdfsConstants.MEMORY_STORAGE_POLICY_NAME);
-    return p != null && stat.getStoragePolicy() == p.getId();
-  }
 
   private void computePacketChunkSize(int psize, int csize) {
     final int bodySize = psize - PacketHeader.PKT_MAX_HEADER_LEN;
@@ -1728,62 +369,6 @@ public class DFSOutputStream extends FSOutputSummer
     }
   }
 
-  private void queueCurrentPacket() {
-    synchronized (dataQueue) {
-      if (currentPacket == null) return;
-      currentPacket.addTraceParent(Trace.currentSpan());
-      dataQueue.addLast(currentPacket);
-      lastQueuedSeqno = currentPacket.getSeqno();
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Queued packet " + currentPacket.getSeqno());
-      }
-      currentPacket = null;
-      dataQueue.notifyAll();
-    }
-  }
-
-  private void waitAndQueueCurrentPacket() throws IOException {
-    synchronized (dataQueue) {
-      try {
-      // If queue is full, then wait till we have enough space
-        boolean firstWait = true;
-        try {
-          while (!isClosed() && dataQueue.size() + ackQueue.size() >
-              dfsClient.getConf().writeMaxPackets) {
-            if (firstWait) {
-              Span span = Trace.currentSpan();
-              if (span != null) {
-                span.addTimelineAnnotation("dataQueue.wait");
-              }
-              firstWait = false;
-            }
-            try {
-              dataQueue.wait();
-            } catch (InterruptedException e) {
-              // If we get interrupted while waiting to queue data, we still need to get rid
-              // of the current packet. This is because we have an invariant that if
-              // currentPacket gets full, it will get queued before the next writeChunk.
-              //
-              // Rather than wait around for space in the queue, we should instead try to
-              // return to the caller as soon as possible, even though we slightly overrun
-              // the MAX_PACKETS length.
-              Thread.currentThread().interrupt();
-              break;
-            }
-          }
-        } finally {
-          Span span = Trace.currentSpan();
-          if ((span != null) && (!firstWait)) {
-            span.addTimelineAnnotation("end.wait");
-          }
-        }
-        checkClosed();
-        queueCurrentPacket();
-      } catch (ClosedChannelException e) {
-      }
-    }
-  }
-
   // @see FSOutputSummer#writeChunk()
   @Override
   protected synchronized void writeChunk(byte[] b, int offset, int len,
@@ -1814,57 +399,62 @@ public class DFSOutputStream extends FSOutputSummer
 
     if (currentPacket == null) {
       currentPacket = createPacket(packetSize, chunksPerPacket, 
-          bytesCurBlock, currentSeqno++, false);
+          streamer.getBytesCurBlock(), streamer.getAndIncCurrentSeqno(), false);
       if (DFSClient.LOG.isDebugEnabled()) {
         DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" + 
             currentPacket.getSeqno() +
             ", src=" + src +
             ", packetSize=" + packetSize +
             ", chunksPerPacket=" + chunksPerPacket +
-            ", bytesCurBlock=" + bytesCurBlock);
+            ", bytesCurBlock=" + streamer.getBytesCurBlock());
       }
     }
 
     currentPacket.writeChecksum(checksum, ckoff, cklen);
     currentPacket.writeData(b, offset, len);
     currentPacket.incNumChunks();
-    bytesCurBlock += len;
+    streamer.incBytesCurBlock(len);
 
     // If packet is full, enqueue it for transmission
     //
     if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
-        bytesCurBlock == blockSize) {
+        streamer.getBytesCurBlock() == blockSize) {
       if (DFSClient.LOG.isDebugEnabled()) {
         DFSClient.LOG.debug("DFSClient writeChunk packet full seqno=" +
             currentPacket.getSeqno() +
             ", src=" + src +
-            ", bytesCurBlock=" + bytesCurBlock +
+            ", bytesCurBlock=" + streamer.getBytesCurBlock() +
             ", blockSize=" + blockSize +
-            ", appendChunk=" + appendChunk);
+            ", appendChunk=" + streamer.getAppendChunk());
       }
-      waitAndQueueCurrentPacket();
+      streamer.waitAndQueuePacket(currentPacket);
+      currentPacket = null;
 
       // If the reopened file did not end at chunk boundary and the above
       // write filled up its partial chunk. Tell the summer to generate full 
       // crc chunks from now on.
-      if (appendChunk && bytesCurBlock%bytesPerChecksum == 0) {
-        appendChunk = false;
+      if (streamer.getAppendChunk() &&
+          streamer.getBytesCurBlock() % bytesPerChecksum == 0) {
+        streamer.setAppendChunk(false);
         resetChecksumBufSize();
       }
 
-      if (!appendChunk) {
-        int psize = Math.min((int)(blockSize-bytesCurBlock), dfsClient.getConf().writePacketSize);
+      if (!streamer.getAppendChunk()) {
+        int psize = Math.min((int)(blockSize-streamer.getBytesCurBlock()),
+            dfsClient.getConf().writePacketSize);
         computePacketChunkSize(psize, bytesPerChecksum);
       }
       //
       // if encountering a block boundary, send an empty packet to 
       // indicate the end of block and reset bytesCurBlock.
       //
-      if (bytesCurBlock == blockSize) {
-        currentPacket = createPacket(0, 0, bytesCurBlock, currentSeqno++, true);
+      if (streamer.getBytesCurBlock() == blockSize) {
+        currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
+            streamer.getAndIncCurrentSeqno(), true);
         currentPacket.setSyncBlock(shouldSyncBlock);
-        waitAndQueueCurrentPacket();
-        bytesCurBlock = 0;
+        streamer.waitAndQueuePacket(currentPacket);
+        currentPacket = null;
+        streamer.setBytesCurBlock(0);
         lastFlushOffset = 0;
       }
     }
@@ -1954,30 +544,30 @@ public class DFSOutputStream extends FSOutputSummer
 
         if (DFSClient.LOG.isDebugEnabled()) {
           DFSClient.LOG.debug("DFSClient flush(): "
-              + " bytesCurBlock=" + bytesCurBlock
+              + " bytesCurBlock=" + streamer.getBytesCurBlock()
               + " lastFlushOffset=" + lastFlushOffset
               + " createNewBlock=" + endBlock);
         }
         // Flush only if we haven't already flushed till this offset.
-        if (lastFlushOffset != bytesCurBlock) {
-          assert bytesCurBlock > lastFlushOffset;
+        if (lastFlushOffset != streamer.getBytesCurBlock()) {
+          assert streamer.getBytesCurBlock() > lastFlushOffset;
           // record the valid offset of this flush
-          lastFlushOffset = bytesCurBlock;
+          lastFlushOffset = streamer.getBytesCurBlock();
           if (isSync && currentPacket == null && !endBlock) {
             // Nothing to send right now,
             // but sync was requested.
             // Send an empty packet if we do not end the block right now
             currentPacket = createPacket(packetSize, chunksPerPacket,
-                bytesCurBlock, currentSeqno++, false);
+                streamer.getBytesCurBlock(), streamer.getAndIncCurrentSeqno(), false);
           }
         } else {
-          if (isSync && bytesCurBlock > 0 && !endBlock) {
+          if (isSync && streamer.getBytesCurBlock() > 0 && !endBlock) {
             // Nothing to send right now,
             // and the block was partially written,
             // and sync was requested.
             // So send an empty sync packet if we do not end the block right now
             currentPacket = createPacket(packetSize, chunksPerPacket,
-                bytesCurBlock, currentSeqno++, false);
+                streamer.getBytesCurBlock(), streamer.getAndIncCurrentSeqno(), false);
           } else if (currentPacket != null) {
             // just discard the current packet since it is already been sent.
             currentPacket.releaseBuffer(byteArrayManager);
@@ -1986,39 +576,42 @@ public class DFSOutputStream extends FSOutputSummer
         }
         if (currentPacket != null) {
           currentPacket.setSyncBlock(isSync);
-          waitAndQueueCurrentPacket();          
+          streamer.waitAndQueuePacket(currentPacket);
+          currentPacket = null;
         }
-        if (endBlock && bytesCurBlock > 0) {
+        if (endBlock && streamer.getBytesCurBlock() > 0) {
           // Need to end the current block, thus send an empty packet to
           // indicate this is the end of the block and reset bytesCurBlock
-          currentPacket = createPacket(0, 0, bytesCurBlock, currentSeqno++, true);
+          currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
+              streamer.getAndIncCurrentSeqno(), true);
           currentPacket.setSyncBlock(shouldSyncBlock || isSync);
-          waitAndQueueCurrentPacket();
-          bytesCurBlock = 0;
+          streamer.waitAndQueuePacket(currentPacket);
+          currentPacket = null;
+          streamer.setBytesCurBlock(0);
           lastFlushOffset = 0;
         } else {
           // Restore state of stream. Record the last flush offset
           // of the last full chunk that was flushed.
-          bytesCurBlock -= numKept;
+          streamer.setBytesCurBlock(streamer.getBytesCurBlock() - numKept);
         }
 
-        toWaitFor = lastQueuedSeqno;
+        toWaitFor = streamer.getLastQueuedSeqno();
       } // end synchronized
 
-      waitForAckedSeqno(toWaitFor);
+      streamer.waitForAckedSeqno(toWaitFor);
 
       // update the block length first time irrespective of flag
-      if (updateLength || persistBlocks.get()) {
+      if (updateLength || streamer.getPersistBlocks().get()) {
         synchronized (this) {
-          if (streamer != null && streamer.block != null) {
-            lastBlockLength = streamer.block.getNumBytes();
+          if (!streamer.streamerClosed() && streamer.getBlock() != null) {
+            lastBlockLength = streamer.getBlock().getNumBytes();
           }
         }
       }
       // If 1) any new blocks were allocated since the last flush, or 2) to
       // update length in NN is required, then persist block locations on
       // namenode.
-      if (persistBlocks.getAndSet(false) || updateLength) {
+      if (streamer.getPersistBlocks().getAndSet(false) || updateLength) {
         try {
           dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
               lastBlockLength);
@@ -2035,7 +628,7 @@ public class DFSOutputStream extends FSOutputSummer
       }
 
       synchronized(this) {
-        if (streamer != null) {
+        if (!streamer.streamerClosed()) {
           streamer.setHflush();
         }
       }
@@ -2048,7 +641,7 @@ public class DFSOutputStream extends FSOutputSummer
       DFSClient.LOG.warn("Error while syncing", e);
       synchronized (this) {
         if (!isClosed()) {
-          lastException.set(new IOException("IOException flush: " + e));
+          streamer.getLastException().set(new IOException("IOException flush: " + e));
           closeThreads(true);
         }
       }
@@ -2073,7 +666,7 @@ public class DFSOutputStream extends FSOutputSummer
   public synchronized int getCurrentBlockReplication() throws IOException {
     dfsClient.checkOpen();
     checkClosed();
-    if (streamer == null) {
+    if (streamer.streamerClosed()) {
       return blockReplication; // no pipeline, return repl factor of file
     }
     DatanodeInfo[] currentNodes = streamer.getNodes();
@@ -2095,47 +688,12 @@ public class DFSOutputStream extends FSOutputSummer
       //
       // If there is data in the current buffer, send it across
       //
-      queueCurrentPacket();
-      toWaitFor = lastQueuedSeqno;
+      streamer.queuePacket(currentPacket);
+      currentPacket = null;
+      toWaitFor = streamer.getLastQueuedSeqno();
     }
 
-    waitForAckedSeqno(toWaitFor);
-  }
-
-  private void waitForAckedSeqno(long seqno) throws IOException {
-    TraceScope scope = Trace.startSpan("waitForAckedSeqno", Sampler.NEVER);
-    try {
-      if (DFSClient.LOG.isDebugEnabled()) {
-        DFSClient.LOG.debug("Waiting for ack for: " + seqno);
-      }
-      long begin = Time.monotonicNow();
-      try {
-        synchronized (dataQueue) {
-          while (!isClosed()) {
-            checkClosed();
-            if (lastAckedSeqno >= seqno) {
-              break;
-            }
-            try {
-              dataQueue.wait(1000); // when we receive an ack, we notify on
-              // dataQueue
-            } catch (InterruptedException ie) {
-              throw new InterruptedIOException(
-                  "Interrupted while waiting for data to be acknowledged by pipeline");
-            }
-          }
-        }
-        checkClosed();
-      } catch (ClosedChannelException e) {
-      }
-      long duration = Time.monotonicNow() - begin;
-      if (duration > dfsclientSlowLogThresholdMs) {
-        DFSClient.LOG.warn("Slow waitForAckedSeqno took " + duration
-            + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
-      }
-    } finally {
-      scope.close();
-    }
+    streamer.waitForAckedSeqno(toWaitFor);
   }
 
   private synchronized void start() {
@@ -2157,22 +715,12 @@ public class DFSOutputStream extends FSOutputSummer
   }
 
   boolean isClosed() {
-    return closed;
+    return closed || streamer.streamerClosed();
   }
 
   void setClosed() {
     closed = true;
-    synchronized (dataQueue) {
-      releaseBuffer(dataQueue, byteArrayManager);
-      releaseBuffer(ackQueue, byteArrayManager);
-    }
-  }
-  
-  private static void releaseBuffer(List<DFSPacket> packets, ByteArrayManager bam) {
-    for (DFSPacket p : packets) {
-      p.releaseBuffer(bam);
-    }
-    packets.clear();
+    streamer.release();
   }
 
   // shutdown datastreamer and responseprocessor threads.
@@ -2181,14 +729,11 @@ public class DFSOutputStream extends FSOutputSummer
     try {
       streamer.close(force);
       streamer.join();
-      if (s != null) {
-        s.close();
-      }
+      streamer.closeSocket();
     } catch (InterruptedException e) {
       throw new IOException("Failed to shutdown streamer");
     } finally {
-      streamer = null;
-      s = null;
+      streamer.setSocketToNull();
       setClosed();
     }
   }
@@ -2210,7 +755,7 @@ public class DFSOutputStream extends FSOutputSummer
 
   private synchronized void closeImpl() throws IOException {
     if (isClosed()) {
-      IOException e = lastException.getAndSet(null);
+      IOException e = streamer.getLastException().getAndSet(null);
       if (e == null)
         return;
       else
@@ -2221,12 +766,14 @@ public class DFSOutputStream extends FSOutputSummer
       flushBuffer();       // flush from all upper layers
 
       if (currentPacket != null) { 
-        waitAndQueueCurrentPacket();
+        streamer.waitAndQueuePacket(currentPacket);
+        currentPacket = null;
       }
 
-      if (bytesCurBlock != 0) {
+      if (streamer.getBytesCurBlock() != 0) {
         // send an empty packet to mark the end of the block
-        currentPacket = createPacket(0, 0, bytesCurBlock, currentSeqno++, true);
+        currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
+            streamer.getAndIncCurrentSeqno(), true);
         currentPacket.setSyncBlock(shouldSyncBlock);
       }
 
@@ -2261,7 +808,7 @@ public class DFSOutputStream extends FSOutputSummer
       if (!fileComplete) {
         final int hdfsTimeout = dfsClient.getHdfsTimeout();
         if (!dfsClient.clientRunning
-            || (hdfsTimeout > 0 
+            || (hdfsTimeout > 0
                 && localstart + hdfsTimeout < Time.monotonicNow())) {
             String msg = "Unable to close file because dfsclient " +
                           " was unable to contact the HDFS servers." +
@@ -2290,7 +837,7 @@ public class DFSOutputStream extends FSOutputSummer
 
   @VisibleForTesting
   public void setArtificialSlowdown(long period) {
-    artificialSlowdown = period;
+    streamer.setArtificialSlowdown(period);
   }
 
   @VisibleForTesting
@@ -2299,10 +846,6 @@ public class DFSOutputStream extends FSOutputSummer
     packetSize = (bytesPerChecksum + getChecksumSize()) * chunksPerPacket;
   }
 
-  synchronized void setTestFilename(String newname) {
-    src = newname;
-  }
-
   /**
    * Returns the size of a file as it was when this stream was opened
    */
@@ -2345,9 +888,4 @@ public class DFSOutputStream extends FSOutputSummer
   public long getFileId() {
     return fileId;
   }
-
-  private static <T> void arraycopy(T[] srcs, T[] dsts, int skipIndex) {
-    System.arraycopy(srcs, 0, dsts, 0, skipIndex);
-    System.arraycopy(srcs, skipIndex+1, dsts, skipIndex, dsts.length-skipIndex);
-  }
 }


[15/51] [abbrv] hadoop git commit: MAPREDUCE-6286. Amend commit to CHANGES.txt for backport into 2.7.0.

Posted by ka...@apache.org.
MAPREDUCE-6286. Amend commit to CHANGES.txt for backport into 2.7.0.


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

Branch: refs/heads/YARN-2139
Commit: 8770c82acc948bc5127afb1c59072718fd04630c
Parents: 1d5c796
Author: Harsh J <ha...@cloudera.com>
Authored: Sun Mar 22 10:15:52 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Sun Mar 22 10:15:52 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8770c82a/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index e98aacd..b75d8aa 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -292,10 +292,6 @@ Release 2.8.0 - UNRELEASED
     MAPREDUCE-5448. MapFileOutputFormat#getReaders bug with hidden
     files/folders. (Maysam Yabandeh via harsh)
 
-    MAPREDUCE-6286. A typo in HistoryViewer makes some code useless, which
-    causes counter limits are not reset correctly.
-    (Zhihai Xu via harsh)
-
     MAPREDUCE-6213. NullPointerException caused by job history server addr not
     resolvable. (Peng Zhang via harsh)
 
@@ -398,6 +394,10 @@ Release 2.7.0 - UNRELEASED
 
   BUG FIXES
 
+    MAPREDUCE-6286. A typo in HistoryViewer makes some code useless, which
+    causes counter limits are not reset correctly.
+    (Zhihai Xu via harsh)
+
     MAPREDUCE-6210. Use getApplicationAttemptId() instead of getApplicationId()
     for logging AttemptId in RMContainerAllocator.java (Leitao Guo via aajisaka)
 


[28/51] [abbrv] hadoop git commit: HDFS-7960. The full block report should prune zombie storages even if they're not empty. Contributed by Colin McCabe and Eddy Xu.

Posted by ka...@apache.org.
HDFS-7960. The full block report should prune zombie storages even if they're not empty. Contributed by Colin McCabe and Eddy Xu.


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

Branch: refs/heads/YARN-2139
Commit: 50ee8f4e67a66aa77c5359182f61f3e951844db6
Parents: d7e3c33
Author: Andrew Wang <wa...@apache.org>
Authored: Mon Mar 23 22:00:34 2015 -0700
Committer: Andrew Wang <wa...@apache.org>
Committed: Mon Mar 23 22:00:34 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../DatanodeProtocolClientSideTranslatorPB.java |   5 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   4 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  15 +++
 .../server/blockmanagement/BlockManager.java    |  53 +++++++-
 .../blockmanagement/DatanodeDescriptor.java     |  51 ++++++-
 .../blockmanagement/DatanodeStorageInfo.java    |  13 +-
 .../hdfs/server/datanode/BPServiceActor.java    |  34 +++--
 .../hdfs/server/namenode/NameNodeRpcServer.java |  11 +-
 .../server/protocol/BlockReportContext.java     |  52 +++++++
 .../hdfs/server/protocol/DatanodeProtocol.java  |  10 +-
 .../src/main/proto/DatanodeProtocol.proto       |  14 ++
 .../hdfs/protocol/TestBlockListAsLongs.java     |   7 +-
 .../blockmanagement/TestBlockManager.java       |   8 +-
 .../TestNameNodePrunesMissingStorages.java      | 135 ++++++++++++++++++-
 .../server/datanode/BlockReportTestBase.java    |   4 +-
 .../server/datanode/TestBPOfferService.java     |  10 +-
 .../TestBlockHasMultipleReplicasOnSameDN.java   |   4 +-
 .../datanode/TestDataNodeVolumeFailure.java     |   3 +-
 .../TestDatanodeProtocolRetryPolicy.java        |   4 +-
 ...TestDnRespectsBlockReportSplitThreshold.java |   7 +-
 .../TestNNHandlesBlockReportPerStorage.java     |   7 +-
 .../TestNNHandlesCombinedBlockReport.java       |   4 +-
 .../server/datanode/TestTriggerBlockReport.java |   7 +-
 .../server/namenode/NNThroughputBenchmark.java  |   9 +-
 .../hdfs/server/namenode/TestDeadDatanode.java  |   4 +-
 .../hdfs/server/namenode/ha/TestDNFencing.java  |   4 +-
 27 files changed, 433 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 d2891e3..3dd5fb3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1241,6 +1241,9 @@ Release 2.7.0 - UNRELEASED
     provided by the client is larger than the one stored in the datanode.
     (Brahma Reddy Battula via szetszwo)
 
+    HDFS-7960. The full block report should prune zombie storages even if
+    they're not empty. (cmccabe and Eddy Xu via wang)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 c4003f1..825e835 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
@@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReportBadBlo
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageBlockReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.StorageReceivedDeletedBlocksProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.VersionRequestProto;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -169,7 +170,8 @@ public class DatanodeProtocolClientSideTranslatorPB implements
 
   @Override
   public DatanodeCommand blockReport(DatanodeRegistration registration,
-      String poolId, StorageBlockReport[] reports) throws IOException {
+      String poolId, StorageBlockReport[] reports, BlockReportContext context)
+        throws IOException {
     BlockReportRequestProto.Builder builder = BlockReportRequestProto
         .newBuilder().setRegistration(PBHelper.convert(registration))
         .setBlockPoolId(poolId);
@@ -191,6 +193,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       }
       builder.addReports(reportBuilder.build());
     }
+    builder.setContext(PBHelper.convert(context));
     BlockReportResponseProto resp;
     try {
       resp = rpcProxy.blockReport(NULL_CONTROLLER, builder.build());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 e18081f..873eb6d 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
@@ -161,7 +161,9 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     }
     try {
       cmd = impl.blockReport(PBHelper.convert(request.getRegistration()),
-          request.getBlockPoolId(), report);
+          request.getBlockPoolId(), report,
+          request.hasContext() ?
+              PBHelper.convert(request.getContext()) : null);
     } catch (IOException e) {
       throw new ServiceException(e);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 fad1d2c..b841850 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
@@ -111,6 +111,7 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rollin
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
@@ -123,6 +124,7 @@ import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.NNHAStatusHe
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.ReceivedDeletedBlockInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.RegisterCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.VolumeFailureSummaryProto;
+import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockReportContextProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockKeyProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
@@ -194,6 +196,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockIdCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand;
 import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock;
+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.CheckpointCommand;
@@ -3009,4 +3012,16 @@ public class PBHelper {
     return targetPinnings;
   }
 
+  public static BlockReportContext convert(BlockReportContextProto proto) {
+    return new BlockReportContext(proto.getTotalRpcs(),
+        proto.getCurRpc(), proto.getId());
+  }
+
+  public static BlockReportContextProto convert(BlockReportContext context) {
+    return BlockReportContextProto.newBuilder().
+        setTotalRpcs(context.getTotalRpcs()).
+        setCurRpc(context.getCurRpc()).
+        setId(context.getReportId()).
+        build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 674c0ea..91cfead 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
@@ -69,6 +69,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.Namesystem;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+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;
@@ -1770,7 +1771,8 @@ public class BlockManager {
    */
   public boolean processReport(final DatanodeID nodeID,
       final DatanodeStorage storage,
-      final BlockListAsLongs newReport) throws IOException {
+      final BlockListAsLongs newReport, BlockReportContext context,
+      boolean lastStorageInRpc) throws IOException {
     namesystem.writeLock();
     final long startTime = Time.monotonicNow(); //after acquiring write lock
     final long endTime;
@@ -1809,6 +1811,29 @@ public class BlockManager {
       }
       
       storageInfo.receivedBlockReport();
+      if (context != null) {
+        storageInfo.setLastBlockReportId(context.getReportId());
+        if (lastStorageInRpc) {
+          int rpcsSeen = node.updateBlockReportContext(context);
+          if (rpcsSeen >= context.getTotalRpcs()) {
+            List<DatanodeStorageInfo> zombies = node.removeZombieStorages();
+            if (zombies.isEmpty()) {
+              LOG.debug("processReport 0x{}: no zombie storages found.",
+                  Long.toHexString(context.getReportId()));
+            } else {
+              for (DatanodeStorageInfo zombie : zombies) {
+                removeZombieReplicas(context, zombie);
+              }
+            }
+            node.clearBlockReportContext();
+          } else {
+            LOG.debug("processReport 0x{}: {} more RPCs remaining in this " +
+                    "report.", Long.toHexString(context.getReportId()),
+                (context.getTotalRpcs() - rpcsSeen)
+            );
+          }
+        }
+      }
     } finally {
       endTime = Time.monotonicNow();
       namesystem.writeUnlock();
@@ -1833,6 +1858,32 @@ public class BlockManager {
     return !node.hasStaleStorages();
   }
 
+  private void removeZombieReplicas(BlockReportContext context,
+      DatanodeStorageInfo zombie) {
+    LOG.warn("processReport 0x{}: removing zombie storage {}, which no " +
+             "longer exists on the DataNode.",
+              Long.toHexString(context.getReportId()), zombie.getStorageID());
+    assert(namesystem.hasWriteLock());
+    Iterator<BlockInfoContiguous> iter = zombie.getBlockIterator();
+    int prevBlocks = zombie.numBlocks();
+    while (iter.hasNext()) {
+      BlockInfoContiguous block = iter.next();
+      // We assume that a block can be on only one storage in a DataNode.
+      // That's why we pass in the DatanodeDescriptor rather than the
+      // DatanodeStorageInfo.
+      // TODO: remove this assumption in case we want to put a block on
+      // more than one storage on a datanode (and because it's a difficult
+      // assumption to really enforce)
+      removeStoredBlock(block, zombie.getDatanodeDescriptor());
+      invalidateBlocks.remove(zombie.getDatanodeDescriptor(), block);
+    }
+    assert(zombie.numBlocks() == 0);
+    LOG.warn("processReport 0x{}: removed {} replicas from storage {}, " +
+            "which no longer exists on the DataNode.",
+            Long.toHexString(context.getReportId()), prevBlocks,
+            zombie.getStorageID());
+  }
+
   /**
    * Rescan the list of blocks which were previously postponed.
    */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 3f143e7..d0d7a72 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
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import java.util.ArrayList;
+import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -31,6 +32,7 @@ import java.util.Set;
 
 import com.google.common.annotations.VisibleForTesting;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
@@ -40,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
@@ -62,7 +65,25 @@ public class DatanodeDescriptor extends DatanodeInfo {
   // Stores status of decommissioning.
   // If node is not decommissioning, do not use this object for anything.
   public final DecommissioningStatus decommissioningStatus = new DecommissioningStatus();
-  
+
+  private long curBlockReportId = 0;
+
+  private BitSet curBlockReportRpcsSeen = null;
+
+  public int updateBlockReportContext(BlockReportContext context) {
+    if (curBlockReportId != context.getReportId()) {
+      curBlockReportId = context.getReportId();
+      curBlockReportRpcsSeen = new BitSet(context.getTotalRpcs());
+    }
+    curBlockReportRpcsSeen.set(context.getCurRpc());
+    return curBlockReportRpcsSeen.cardinality();
+  }
+
+  public void clearBlockReportContext() {
+    curBlockReportId = 0;
+    curBlockReportRpcsSeen = null;
+  }
+
   /** Block and targets pair */
   @InterfaceAudience.Private
   @InterfaceStability.Evolving
@@ -282,6 +303,34 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
+  static final private List<DatanodeStorageInfo> EMPTY_STORAGE_INFO_LIST =
+      ImmutableList.of();
+
+  List<DatanodeStorageInfo> removeZombieStorages() {
+    List<DatanodeStorageInfo> zombies = null;
+    synchronized (storageMap) {
+      Iterator<Map.Entry<String, DatanodeStorageInfo>> iter =
+          storageMap.entrySet().iterator();
+      while (iter.hasNext()) {
+        Map.Entry<String, DatanodeStorageInfo> entry = iter.next();
+        DatanodeStorageInfo storageInfo = entry.getValue();
+        if (storageInfo.getLastBlockReportId() != curBlockReportId) {
+          LOG.info(storageInfo.getStorageID() + " had lastBlockReportId 0x" +
+              Long.toHexString(storageInfo.getLastBlockReportId()) +
+              ", but curBlockReportId = 0x" +
+              Long.toHexString(curBlockReportId));
+          iter.remove();
+          if (zombies == null) {
+            zombies = new LinkedList<DatanodeStorageInfo>();
+          }
+          zombies.add(storageInfo);
+        }
+        storageInfo.setLastBlockReportId(0);
+      }
+    }
+    return zombies == null ? EMPTY_STORAGE_INFO_LIST : zombies;
+  }
+
   /**
    * Remove block from the list of blocks belonging to the data-node. Remove
    * data-node from the block.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
index c4612a3..be16a87 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
@@ -115,6 +115,9 @@ public class DatanodeStorageInfo {
   private volatile BlockInfoContiguous blockList = null;
   private int numBlocks = 0;
 
+  // The ID of the last full block report which updated this storage.
+  private long lastBlockReportId = 0;
+
   /** The number of block reports received */
   private int blockReportCount = 0;
 
@@ -178,7 +181,15 @@ public class DatanodeStorageInfo {
     this.remaining = remaining;
     this.blockPoolUsed = blockPoolUsed;
   }
-  
+
+  long getLastBlockReportId() {
+    return lastBlockReportId;
+  }
+
+  void setLastBlockReportId(long lastBlockReportId) {
+    this.lastBlockReportId = lastBlockReportId;
+  }
+
   State getState() {
     return this.state;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 90f2fe6..10cce45 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
@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -434,6 +435,17 @@ class BPServiceActor implements Runnable {
     return sendImmediateIBR;
   }
 
+  private long prevBlockReportId = 0;
+
+  private long generateUniqueBlockReportId() {
+    long id = System.nanoTime();
+    if (id <= prevBlockReportId) {
+      id = prevBlockReportId + 1;
+    }
+    prevBlockReportId = id;
+    return id;
+  }
+
   /**
    * Report the list blocks to the Namenode
    * @return DatanodeCommands returned by the NN. May be null.
@@ -476,11 +488,13 @@ class BPServiceActor implements Runnable {
     int numRPCs = 0;
     boolean success = false;
     long brSendStartTime = monotonicNow();
+    long reportId = generateUniqueBlockReportId();
     try {
       if (totalBlockCount < dnConf.blockReportSplitThreshold) {
         // Below split threshold, send all reports in a single message.
         DatanodeCommand cmd = bpNamenode.blockReport(
-            bpRegistration, bpos.getBlockPoolId(), reports);
+            bpRegistration, bpos.getBlockPoolId(), reports,
+              new BlockReportContext(1, 0, reportId));
         numRPCs = 1;
         numReportsSent = reports.length;
         if (cmd != null) {
@@ -488,10 +502,11 @@ class BPServiceActor implements Runnable {
         }
       } else {
         // Send one block report per message.
-        for (StorageBlockReport report : reports) {
-          StorageBlockReport singleReport[] = { report };
+        for (int r = 0; r < reports.length; r++) {
+          StorageBlockReport singleReport[] = { reports[r] };
           DatanodeCommand cmd = bpNamenode.blockReport(
-              bpRegistration, bpos.getBlockPoolId(), singleReport);
+              bpRegistration, bpos.getBlockPoolId(), singleReport,
+              new BlockReportContext(reports.length, r, reportId));
           numReportsSent++;
           numRPCs++;
           if (cmd != null) {
@@ -507,11 +522,12 @@ class BPServiceActor implements Runnable {
       dn.getMetrics().addBlockReport(brSendCost);
       final int nCmds = cmds.size();
       LOG.info((success ? "S" : "Uns") +
-          "uccessfully sent " + numReportsSent +
-          " of " + reports.length +
-          " blockreports for " + totalBlockCount +
-          " total blocks using " + numRPCs +
-          " RPCs. This took " + brCreateCost +
+          "uccessfully sent block report 0x" +
+          Long.toHexString(reportId) + ",  containing " + reports.length +
+          " storage report(s), of which we sent " + numReportsSent + "." +
+          " The reports had " + totalBlockCount +
+          " total blocks and used " + numRPCs +
+          " RPC(s). This took " + brCreateCost +
           " msec to generate and " + brSendCost +
           " msecs for RPC and NN processing." +
           " Got back " +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 059bd28..1788335 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
@@ -121,6 +121,7 @@ import org.apache.hadoop.hdfs.server.common.IncorrectVersionException;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.server.namenode.metrics.NameNodeMetrics;
 import org.apache.hadoop.hdfs.server.namenode.web.resources.NamenodeWebHdfsMethods;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
@@ -1292,7 +1293,8 @@ class NameNodeRpcServer implements NamenodeProtocols {
 
   @Override // DatanodeProtocol
   public DatanodeCommand blockReport(DatanodeRegistration nodeReg,
-      String poolId, StorageBlockReport[] reports) throws IOException {
+        String poolId, StorageBlockReport[] reports,
+        BlockReportContext context) throws IOException {
     checkNNStartup();
     verifyRequest(nodeReg);
     if(blockStateChangeLog.isDebugEnabled()) {
@@ -1301,14 +1303,15 @@ class NameNodeRpcServer implements NamenodeProtocols {
     }
     final BlockManager bm = namesystem.getBlockManager(); 
     boolean noStaleStorages = false;
-    for(StorageBlockReport r : reports) {
-      final BlockListAsLongs blocks = r.getBlocks();
+    for (int r = 0; r < reports.length; r++) {
+      final BlockListAsLongs blocks = reports[r].getBlocks();
       //
       // BlockManager.processReport accumulates information of prior calls
       // for the same node and storage, so the value returned by the last
       // call of this loop is the final updated value for noStaleStorage.
       //
-      noStaleStorages = bm.processReport(nodeReg, r.getStorage(), blocks);
+      noStaleStorages = bm.processReport(nodeReg, reports[r].getStorage(),
+          blocks, context, (r == reports.length - 1));
       metrics.incrStorageBlockReportOps();
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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
new file mode 100644
index 0000000..a084a81
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockReportContext.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.protocol;
+
+/**
+ * The context of the block report.
+ *
+ * This is a set of fields that the Datanode sends to provide context about a
+ * block report RPC.  The context includes a unique 64-bit ID which
+ * identifies the block report as a whole.  It also includes the total number
+ * of RPCs which this block report is split into, and the index into that
+ * total for the current RPC.
+ */
+public class BlockReportContext {
+  private final int totalRpcs;
+  private final int curRpc;
+  private final long reportId;
+
+  public BlockReportContext(int totalRpcs, int curRpc, long reportId) {
+    this.totalRpcs = totalRpcs;
+    this.curRpc = curRpc;
+    this.reportId = reportId;
+  }
+
+  public int getTotalRpcs() {
+    return totalRpcs;
+  }
+
+  public int getCurRpc() {
+    return curRpc;
+  }
+
+  public long getReportId() {
+    return reportId;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 047de56..a3b6004 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
@@ -23,7 +23,6 @@ import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -128,20 +127,23 @@ public interface DatanodeProtocol {
    *     Each finalized block is represented as 3 longs. Each under-
    *     construction replica is represented as 4 longs.
    *     This is done instead of Block[] to reduce memory used by block reports.
-   *     
+   * @param reports report of blocks per storage
+   * @param context Context information for this block report.
+   *
    * @return - the next command for DN to process.
    * @throws IOException
    */
   @Idempotent
   public DatanodeCommand blockReport(DatanodeRegistration registration,
-      String poolId, StorageBlockReport[] reports) throws IOException;
+            String poolId, StorageBlockReport[] reports,
+            BlockReportContext context) throws IOException;
     
 
   /**
    * Communicates the complete list of locally cached blocks to the NameNode.
    * 
    * This method is similar to
-   * {@link #blockReport(DatanodeRegistration, String, StorageBlockReport[])},
+   * {@link #blockReport(DatanodeRegistration, String, StorageBlockReport[], BlockReportContext)},
    * which is used to communicated blocks stored on disk.
    *
    * @param            The datanode registration.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 7b3a4a9..3083dc9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
@@ -224,11 +224,25 @@ message HeartbeatResponseProto {
  *                second long represents length
  *                third long represents gen stamp
  *                fourth long (if under construction) represents replica state
+ * context      - An optional field containing information about the context
+ *                of this block report.
  */
 message BlockReportRequestProto {
   required DatanodeRegistrationProto registration = 1;
   required string blockPoolId = 2;
   repeated StorageBlockReportProto reports = 3;
+  optional BlockReportContextProto context = 4;
+}
+
+message BlockReportContextProto  {
+  // The total number of RPCs this block report is broken into.
+  required int32 totalRpcs = 1;
+
+  // The index of the current RPC (zero-based)
+  required int32 curRpc = 2;
+
+  // The unique 64-bit ID of this block report
+  required int64 id = 3;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 bebde18..f0dab4c 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
@@ -47,6 +47,7 @@ import org.apache.hadoop.hdfs.server.datanode.FinalizedReplica;
 import org.apache.hadoop.hdfs.server.datanode.Replica;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
 import org.apache.hadoop.hdfs.server.datanode.ReplicaWaitingToBeRecovered;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
@@ -219,7 +220,8 @@ public class TestBlockListAsLongs {
     // check DN sends new-style BR
     request.set(null);
     nsInfo.setCapabilities(Capability.STORAGE_BLOCK_REPORT_BUFFERS.getMask());
-    nn.blockReport(reg, "pool", sbr);
+    nn.blockReport(reg, "pool", sbr,
+        new BlockReportContext(1, 0, System.nanoTime()));
     BlockReportRequestProto proto = request.get();
     assertNotNull(proto);
     assertTrue(proto.getReports(0).getBlocksList().isEmpty());
@@ -228,7 +230,8 @@ public class TestBlockListAsLongs {
     // back up to prior version and check DN sends old-style BR
     request.set(null);
     nsInfo.setCapabilities(Capability.UNKNOWN.getMask());
-    nn.blockReport(reg, "pool", sbr);
+    nn.blockReport(reg, "pool", sbr,
+        new BlockReportContext(1, 0, System.nanoTime()));
     proto = request.get();
     assertNotNull(proto);
     assertFalse(proto.getReports(0).getBlocksList().isEmpty());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 d9ac9e5..707c780 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
@@ -555,12 +555,12 @@ public class TestBlockManager {
     reset(node);
     
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        BlockListAsLongs.EMPTY);
+        BlockListAsLongs.EMPTY, null, false);
     assertEquals(1, ds.getBlockReportCount());
     // send block report again, should NOT be processed
     reset(node);
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        BlockListAsLongs.EMPTY);
+        BlockListAsLongs.EMPTY, null, false);
     assertEquals(1, ds.getBlockReportCount());
 
     // re-register as if node restarted, should update existing node
@@ -571,7 +571,7 @@ public class TestBlockManager {
     // send block report, should be processed after restart
     reset(node);
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-                     BlockListAsLongs.EMPTY);
+                     BlockListAsLongs.EMPTY, null, false);
     // Reinitialize as registration with empty storage list pruned
     // node.storageMap.
     ds = node.getStorageInfos()[0];
@@ -600,7 +600,7 @@ public class TestBlockManager {
     reset(node);
     doReturn(1).when(node).numBlocks();
     bm.processReport(node, new DatanodeStorage(ds.getStorageID()),
-        BlockListAsLongs.EMPTY);
+        BlockListAsLongs.EMPTY, null, false);
     assertEquals(1, ds.getBlockReportCount());
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 b67ae7a..4b97d01 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
@@ -18,26 +18,40 @@
 
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
+import com.google.common.base.Supplier;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.commons.math3.stat.inference.TestUtils;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.File;
+import java.io.FileOutputStream;
 import java.io.IOException;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
 
 import static org.hamcrest.core.Is.is;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
 
 
 public class TestNameNodePrunesMissingStorages {
@@ -110,7 +124,9 @@ public class TestNameNodePrunesMissingStorages {
   }
 
   /**
-   * Verify that the NameNode does not prune storages with blocks.
+   * Verify that the NameNode does not prune storages with blocks
+   * simply as a result of a heartbeat being sent missing that storage.
+   *
    * @throws IOException
    */
   @Test (timeout=300000)
@@ -118,4 +134,119 @@ public class TestNameNodePrunesMissingStorages {
     // Run the test with 1 storage, after the text still expect 1 storage.
     runTest(GenericTestUtils.getMethodName(), true, 1, 1);
   }
+
+  /**
+   * Regression test for HDFS-7960.<p/>
+   *
+   * Shutting down a datanode, removing a storage directory, and restarting
+   * the DataNode should not produce zombie storages.
+   */
+  @Test(timeout=300000)
+  public void testRemovingStorageDoesNotProduceZombies() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
+    final int NUM_STORAGES_PER_DN = 2;
+    final MiniDFSCluster cluster = new MiniDFSCluster
+        .Builder(conf).numDataNodes(3)
+        .storagesPerDatanode(NUM_STORAGES_PER_DN)
+        .build();
+    try {
+      cluster.waitActive();
+      for (DataNode dn : cluster.getDataNodes()) {
+        assertEquals(NUM_STORAGES_PER_DN,
+          cluster.getNamesystem().getBlockManager().
+              getDatanodeManager().getDatanode(dn.getDatanodeId()).
+              getStorageInfos().length);
+      }
+      // Create a file which will end up on all 3 datanodes.
+      final Path TEST_PATH = new Path("/foo1");
+      DistributedFileSystem fs = cluster.getFileSystem();
+      DFSTestUtil.createFile(fs, TEST_PATH, 1024, (short) 3, 0xcafecafe);
+      for (DataNode dn : cluster.getDataNodes()) {
+        DataNodeTestUtils.triggerBlockReport(dn);
+      }
+      ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, new Path("/foo1"));
+      cluster.getNamesystem().writeLock();
+      final String storageIdToRemove;
+      String datanodeUuid;
+      // Find the first storage which this block is in.
+      try {
+        Iterator<DatanodeStorageInfo> storageInfoIter =
+            cluster.getNamesystem().getBlockManager().
+                getStorages(block.getLocalBlock()).iterator();
+        assertTrue(storageInfoIter.hasNext());
+        DatanodeStorageInfo info = storageInfoIter.next();
+        storageIdToRemove = info.getStorageID();
+        datanodeUuid = info.getDatanodeDescriptor().getDatanodeUuid();
+      } finally {
+        cluster.getNamesystem().writeUnlock();
+      }
+      // Find the DataNode which holds that first storage.
+      final DataNode datanodeToRemoveStorageFrom;
+      int datanodeToRemoveStorageFromIdx = 0;
+      while (true) {
+        if (datanodeToRemoveStorageFromIdx >= cluster.getDataNodes().size()) {
+          Assert.fail("failed to find datanode with uuid " + datanodeUuid);
+          datanodeToRemoveStorageFrom = null;
+          break;
+        }
+        DataNode dn = cluster.getDataNodes().
+            get(datanodeToRemoveStorageFromIdx);
+        if (dn.getDatanodeUuid().equals(datanodeUuid)) {
+          datanodeToRemoveStorageFrom = dn;
+          break;
+        }
+        datanodeToRemoveStorageFromIdx++;
+      }
+      // Find the volume within the datanode which holds that first storage.
+      List<? extends FsVolumeSpi> volumes =
+          datanodeToRemoveStorageFrom.getFSDataset().getVolumes();
+      assertEquals(NUM_STORAGES_PER_DN, volumes.size());
+      String volumeDirectoryToRemove = null;
+      for (FsVolumeSpi volume : volumes) {
+        if (volume.getStorageID().equals(storageIdToRemove)) {
+          volumeDirectoryToRemove = volume.getBasePath();
+        }
+      }
+      // Shut down the datanode and remove the volume.
+      // Replace the volume directory with a regular file, which will
+      // cause a volume failure.  (If we merely removed the directory,
+      // it would be re-initialized with a new storage ID.)
+      assertNotNull(volumeDirectoryToRemove);
+      datanodeToRemoveStorageFrom.shutdown();
+      FileUtil.fullyDelete(new File(volumeDirectoryToRemove));
+      FileOutputStream fos = new FileOutputStream(volumeDirectoryToRemove);
+      try {
+        fos.write(1);
+      } finally {
+        fos.close();
+      }
+      cluster.restartDataNode(datanodeToRemoveStorageFromIdx);
+      // Wait for the NameNode to remove the storage.
+      LOG.info("waiting for the datanode to remove " + storageIdToRemove);
+      GenericTestUtils.waitFor(new Supplier<Boolean>() {
+        @Override
+        public Boolean get() {
+          final DatanodeDescriptor dnDescriptor =
+              cluster.getNamesystem().getBlockManager().getDatanodeManager().
+                  getDatanode(datanodeToRemoveStorageFrom.getDatanodeUuid());
+          assertNotNull(dnDescriptor);
+          DatanodeStorageInfo[] infos = dnDescriptor.getStorageInfos();
+          for (DatanodeStorageInfo info : infos) {
+            if (info.getStorageID().equals(storageIdToRemove)) {
+              LOG.info("Still found storage " + storageIdToRemove + " on " +
+                  info + ".");
+              return false;
+            }
+          }
+          assertEquals(NUM_STORAGES_PER_DN - 1, infos.length);
+          return true;
+        }
+      }, 10, 30000);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
index de66db5..c4a2d06 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/BlockReportTestBase.java
@@ -53,6 +53,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
@@ -613,7 +614,8 @@ public abstract class BlockReportTestBase {
         .when(spy).blockReport(
           Mockito.<DatanodeRegistration>anyObject(),
           Mockito.anyString(),
-          Mockito.<StorageBlockReport[]>anyObject());
+          Mockito.<StorageBlockReport[]>anyObject(),
+          Mockito.<BlockReportContext>anyObject());
 
       // Force a block report to be generated. The block report will have
       // an RBW replica in it. Wait for the RPC to be sent, but block

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 bc49793..3aa9a7b 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
@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -216,7 +217,8 @@ public class TestBPOfferService {
         .when(mockNN2).blockReport(
             Mockito.<DatanodeRegistration>anyObject(),  
             Mockito.eq(FAKE_BPID),
-            Mockito.<StorageBlockReport[]>anyObject());
+            Mockito.<StorageBlockReport[]>anyObject(),
+            Mockito.<BlockReportContext>anyObject());
 
     bpos.start();
     try {
@@ -406,7 +408,8 @@ public class TestBPOfferService {
           Mockito.verify(mockNN).blockReport(
               Mockito.<DatanodeRegistration>anyObject(),  
               Mockito.eq(FAKE_BPID),
-              Mockito.<StorageBlockReport[]>anyObject());
+              Mockito.<StorageBlockReport[]>anyObject(),
+              Mockito.<BlockReportContext>anyObject());
           return true;
         } catch (Throwable t) {
           LOG.info("waiting on block report: " + t.getMessage());
@@ -431,7 +434,8 @@ public class TestBPOfferService {
           Mockito.verify(mockNN).blockReport(
                   Mockito.<DatanodeRegistration>anyObject(),
                   Mockito.eq(FAKE_BPID),
-                  Mockito.<StorageBlockReport[]>anyObject());
+                  Mockito.<StorageBlockReport[]>anyObject(),
+                  Mockito.<BlockReportContext>anyObject());
           return true;
         } catch (Throwable t) {
           LOG.info("waiting on block report: " + t.getMessage());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 3238d6a..c47209e 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
@@ -28,6 +28,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
@@ -122,7 +123,8 @@ public class TestBlockHasMultipleReplicasOnSameDN {
     }
 
     // Should not assert!
-    cluster.getNameNodeRpc().blockReport(dnReg, bpid, reports);
+    cluster.getNameNodeRpc().blockReport(dnReg, bpid, reports,
+        new BlockReportContext(1, 0, System.nanoTime()));
 
     // 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/50ee8f4e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index 0428b81..41e8d7b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetTestUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -185,7 +186,7 @@ public class TestDataNodeVolumeFailure {
             new StorageBlockReport(dnStorage, blockList);
     }
     
-    cluster.getNameNodeRpc().blockReport(dnR, bpid, reports);
+    cluster.getNameNodeRpc().blockReport(dnR, bpid, reports, null);
 
     // verify number of blocks and files...
     verify(filename, filesize);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 ac7ebc0..cab50b5 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
@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.HeartbeatResponse;
@@ -136,7 +137,8 @@ public class TestDatanodeProtocolRetryPolicy {
           Mockito.verify(mockNN).blockReport(
               Mockito.eq(datanodeRegistration),
               Mockito.eq(POOL_ID),
-              Mockito.<StorageBlockReport[]>anyObject());
+              Mockito.<StorageBlockReport[]>anyObject(),
+              Mockito.<BlockReportContext>anyObject());
           return true;
         } catch (Throwable t) {
           LOG.info("waiting on block report: " + t.getMessage());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
index a5e4d4e..aadd9b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDnRespectsBlockReportSplitThreshold.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hdfs.*;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY;
@@ -133,7 +134,7 @@ public class TestDnRespectsBlockReportSplitThreshold {
     Mockito.verify(nnSpy, times(cluster.getStoragesPerDatanode())).blockReport(
         any(DatanodeRegistration.class),
         anyString(),
-        captor.capture());
+        captor.capture(),  Mockito.<BlockReportContext>anyObject());
 
     verifyCapturedArguments(captor, 1, BLOCKS_IN_FILE);
   }
@@ -165,7 +166,7 @@ public class TestDnRespectsBlockReportSplitThreshold {
     Mockito.verify(nnSpy, times(1)).blockReport(
         any(DatanodeRegistration.class),
         anyString(),
-        captor.capture());
+        captor.capture(),  Mockito.<BlockReportContext>anyObject());
 
     verifyCapturedArguments(captor, cluster.getStoragesPerDatanode(), BLOCKS_IN_FILE);
   }
@@ -197,7 +198,7 @@ public class TestDnRespectsBlockReportSplitThreshold {
     Mockito.verify(nnSpy, times(cluster.getStoragesPerDatanode())).blockReport(
         any(DatanodeRegistration.class),
         anyString(),
-        captor.capture());
+        captor.capture(), Mockito.<BlockReportContext>anyObject());
 
     verifyCapturedArguments(captor, 1, BLOCKS_IN_FILE);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 1b03786..b150b0d 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
@@ -20,8 +20,10 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.apache.hadoop.util.Time;
 
 
 /**
@@ -33,10 +35,13 @@ public class TestNNHandlesBlockReportPerStorage extends BlockReportTestBase {
   @Override
   protected void sendBlockReports(DatanodeRegistration dnR, String poolId,
       StorageBlockReport[] reports) throws IOException {
+    int i = 0;
     for (StorageBlockReport report : reports) {
       LOG.info("Sending block report for storage " + report.getStorage().getStorageID());
       StorageBlockReport[] singletonReport = { report };
-      cluster.getNameNodeRpc().blockReport(dnR, poolId, singletonReport);
+      cluster.getNameNodeRpc().blockReport(dnR, poolId, singletonReport,
+          new BlockReportContext(reports.length, i, System.nanoTime()));
+      i++;
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 036b550..dca3c88 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
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs.server.datanode;
 
 import java.io.IOException;
 
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 
@@ -34,6 +35,7 @@ public class TestNNHandlesCombinedBlockReport extends BlockReportTestBase {
   protected void sendBlockReports(DatanodeRegistration dnR, String poolId,
                                   StorageBlockReport[] reports) throws IOException {
     LOG.info("Sending combined block reports for " + dnR);
-    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports);
+    cluster.getNameNodeRpc().blockReport(dnR, poolId, reports,
+        new BlockReportContext(1, 0, System.nanoTime()));
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java
index efb9d98..3195d7d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestTriggerBlockReport.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.BlockReportOptions;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
@@ -76,7 +77,8 @@ public final class TestTriggerBlockReport {
       Mockito.verify(spy, times(0)).blockReport(
           any(DatanodeRegistration.class),
           anyString(),
-          any(StorageBlockReport[].class));
+          any(StorageBlockReport[].class),
+          Mockito.<BlockReportContext>anyObject());
       Mockito.verify(spy, times(1)).blockReceivedAndDeleted(
           any(DatanodeRegistration.class),
           anyString(),
@@ -113,7 +115,8 @@ public final class TestTriggerBlockReport {
       Mockito.verify(spy, timeout(60000)).blockReport(
           any(DatanodeRegistration.class),
           anyString(),
-          any(StorageBlockReport[].class));
+          any(StorageBlockReport[].class),
+          Mockito.<BlockReportContext>anyObject());
     }
 
     cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 bc3c6b5..9e24f72 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
@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.protocol.BlockCommand;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -939,7 +940,8 @@ public class NNThroughputBenchmark implements Tool {
           new StorageBlockReport(storage, BlockListAsLongs.EMPTY)
       };
       nameNodeProto.blockReport(dnRegistration, 
-          nameNode.getNamesystem().getBlockPoolId(), reports);
+          nameNode.getNamesystem().getBlockPoolId(), reports,
+              new BlockReportContext(1, 0, System.nanoTime()));
     }
 
     /**
@@ -1184,8 +1186,9 @@ public class NNThroughputBenchmark implements Tool {
       long start = Time.now();
       StorageBlockReport[] report = { new StorageBlockReport(
           dn.storage, dn.getBlockReportList()) };
-      nameNodeProto.blockReport(dn.dnRegistration, nameNode.getNamesystem()
-          .getBlockPoolId(), report);
+      nameNodeProto.blockReport(dn.dnRegistration,
+          nameNode.getNamesystem().getBlockPoolId(), report,
+          new BlockReportContext(1, 0, System.nanoTime()));
       long end = Time.now();
       return end-start;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/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 ee80b33..92c329e 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
@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
@@ -107,7 +108,8 @@ public class TestDeadDatanode {
         new DatanodeStorage(reg.getDatanodeUuid()),
         BlockListAsLongs.EMPTY) };
     try {
-      dnp.blockReport(reg, poolId, report);
+      dnp.blockReport(reg, poolId, report,
+          new BlockReportContext(1, 0, System.nanoTime()));
       fail("Expected IOException is not thrown");
     } catch (IOException ex) {
       // Expected

http://git-wip-us.apache.org/repos/asf/hadoop/blob/50ee8f4e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
index fa7a307..74358bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDNFencing.java
@@ -53,6 +53,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.NameNodeAdapter;
+import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.io.IOUtils;
@@ -547,7 +548,8 @@ public class TestDNFencing {
         .when(spy).blockReport(
           Mockito.<DatanodeRegistration>anyObject(),
           Mockito.anyString(),
-          Mockito.<StorageBlockReport[]>anyObject());
+          Mockito.<StorageBlockReport[]>anyObject(),
+          Mockito.<BlockReportContext>anyObject());
       dn.scheduleAllBlockReport(0);
       delayer.waitForCall();
       


[22/51] [abbrv] hadoop git commit: YARN-3336. FileSystem memory leak in DelegationTokenRenewer.

Posted by ka...@apache.org.
YARN-3336. FileSystem memory leak in DelegationTokenRenewer.


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

Branch: refs/heads/YARN-2139
Commit: 6ca1f12024fd7cec7b01df0f039ca59f3f365dc1
Parents: 7e6f384
Author: cnauroth <cn...@apache.org>
Authored: Mon Mar 23 10:45:50 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Mon Mar 23 10:45:50 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../security/DelegationTokenRenewer.java        | 13 +++++++--
 .../security/TestDelegationTokenRenewer.java    | 30 ++++++++++++++++++--
 3 files changed, 41 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ca1f120/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index e04624e..b90109c 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -819,6 +819,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3384. TestLogAggregationService.verifyContainerLogs fails after
     YARN-2777. (Naganarasimha G R via ozawa)
 
+    YARN-3336. FileSystem memory leak in DelegationTokenRenewer.
+    (Zhihai Xu via cnauroth)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ca1f120/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.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/security/DelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
index cb456d8..2619971 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
@@ -605,6 +605,7 @@ public class DelegationTokenRenewer extends AbstractService {
     rmContext.getSystemCredentialsForApps().put(applicationId, byteBuffer);
   }
 
+  @VisibleForTesting
   protected Token<?>[] obtainSystemTokensForUser(String user,
       final Credentials credentials) throws IOException, InterruptedException {
     // Get new hdfs tokens on behalf of this user
@@ -615,8 +616,16 @@ public class DelegationTokenRenewer extends AbstractService {
         proxyUser.doAs(new PrivilegedExceptionAction<Token<?>[]>() {
           @Override
           public Token<?>[] run() throws Exception {
-            return FileSystem.get(getConfig()).addDelegationTokens(
-              UserGroupInformation.getLoginUser().getUserName(), credentials);
+            FileSystem fs = FileSystem.get(getConfig());
+            try {
+              return fs.addDelegationTokens(
+                  UserGroupInformation.getLoginUser().getUserName(),
+                  credentials);
+            } finally {
+              // Close the FileSystem created by the new proxy user,
+              // So that we don't leave an entry in the FileSystem cache
+              fs.close();
+            }
           }
         });
     return newTokens;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6ca1f120/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.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/security/TestDelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
index 5d31404..99a506a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java
@@ -287,9 +287,16 @@ public class TestDelegationTokenRenewer {
    * exception
    */
   static class MyFS extends DistributedFileSystem {
-    
-    public MyFS() {}
-    public void close() {}
+    private static AtomicInteger instanceCounter = new AtomicInteger();
+    public MyFS() {
+      instanceCounter.incrementAndGet();
+    }
+    public void close() {
+      instanceCounter.decrementAndGet();
+    }
+    public static int getInstanceCounter() {
+      return instanceCounter.get();
+    }
     @Override
     public void initialize(URI uri, Configuration conf) throws IOException {}
     
@@ -299,6 +306,11 @@ public class TestDelegationTokenRenewer {
       LOG.info("Called MYDFS.getdelegationtoken " + result);
       return result;
     }
+
+    public Token<?>[] addDelegationTokens(
+        final String renewer, Credentials credentials) throws IOException {
+      return new Token<?>[0];
+    }
   }
   
   /**
@@ -1022,4 +1034,16 @@ public class TestDelegationTokenRenewer {
     // app2 completes, app1 is still running, check the token is not cancelled
     Assert.assertFalse(Renewer.cancelled);
   }
+
+  // Test FileSystem memory leak in obtainSystemTokensForUser.
+  @Test
+  public void testFSLeakInObtainSystemTokensForUser() throws Exception{
+    Credentials credentials = new Credentials();
+    String user = "test";
+    int oldCounter = MyFS.getInstanceCounter();
+    delegationTokenRenewer.obtainSystemTokensForUser(user, credentials);
+    delegationTokenRenewer.obtainSystemTokensForUser(user, credentials);
+    delegationTokenRenewer.obtainSystemTokensForUser(user, credentials);
+    Assert.assertEquals(oldCounter, MyFS.getInstanceCounter());
+  }
 }


[21/51] [abbrv] hadoop git commit: MAPREDUCE-6242. Progress report log is incredibly excessive in application master. Contributed by Varun Saxena.

Posted by ka...@apache.org.
MAPREDUCE-6242. Progress report log is incredibly excessive in application
master. 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/7e6f384d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7e6f384d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7e6f384d

Branch: refs/heads/YARN-2139
Commit: 7e6f384dd742de21f29e96ee76df5316529c9019
Parents: 36af4a9
Author: Devaraj K <de...@apache.org>
Authored: Mon Mar 23 22:51:20 2015 +0530
Committer: Devaraj K <de...@apache.org>
Committed: Mon Mar 23 22:51:20 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |   3 +
 .../java/org/apache/hadoop/mapred/Task.java     |  13 +-
 .../apache/hadoop/mapreduce/MRJobConfig.java    |   5 +
 .../hadoop/mapred/TestTaskProgressReporter.java | 160 +++++++++++++++++++
 4 files changed, 177 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e6f384d/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 20505b6..b8a2a1c 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -300,6 +300,9 @@ Release 2.8.0 - UNRELEASED
 
     MAPREDUCE-6281. Fix javadoc in Terasort. (Albert Chu via ozawa)
 
+    MAPREDUCE-6242. Progress report log is incredibly excessive in 
+    application master. (Varun Saxena via devaraj)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e6f384d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
index 7fa5d02..bf5ca22 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/Task.java
@@ -229,6 +229,11 @@ abstract public class Task implements Writable, Configurable {
     gcUpdater = new GcTimeUpdater();
   }
 
+  @VisibleForTesting
+  void setTaskDone() {
+    taskDone.set(true);
+  }
+
   ////////////////////////////////////////////
   // Accessors
   ////////////////////////////////////////////
@@ -536,9 +541,6 @@ abstract public class Task implements Writable, Configurable {
   public abstract void run(JobConf job, TaskUmbilicalProtocol umbilical)
     throws IOException, ClassNotFoundException, InterruptedException;
 
-  /** The number of milliseconds between progress reports. */
-  public static final int PROGRESS_INTERVAL = 3000;
-
   private transient Progress taskProgress = new Progress();
 
   // Current counters
@@ -714,6 +716,9 @@ abstract public class Task implements Writable, Configurable {
       int remainingRetries = MAX_RETRIES;
       // get current flag value and reset it as well
       boolean sendProgress = resetProgressFlag();
+      long taskProgressInterval =
+          conf.getLong(MRJobConfig.TASK_PROGRESS_REPORT_INTERVAL,
+                       MRJobConfig.DEFAULT_TASK_PROGRESS_REPORT_INTERVAL);
       while (!taskDone.get()) {
         synchronized (lock) {
           done = false;
@@ -726,7 +731,7 @@ abstract public class Task implements Writable, Configurable {
             if (taskDone.get()) {
               break;
             }
-            lock.wait(PROGRESS_INTERVAL);
+            lock.wait(taskProgressInterval);
           }
           if (taskDone.get()) {
             break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e6f384d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
index f0a6ddf..947c814 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java
@@ -49,6 +49,11 @@ public interface MRJobConfig {
 
   public static final String TASK_CLEANUP_NEEDED = "mapreduce.job.committer.task.cleanup.needed";
 
+  public static final String TASK_PROGRESS_REPORT_INTERVAL =
+      "mapreduce.task.progress-report.interval";
+  /** The number of milliseconds between progress reports. */
+  public static final int DEFAULT_TASK_PROGRESS_REPORT_INTERVAL = 3000;
+
   public static final String JAR = "mapreduce.job.jar";
 
   public static final String ID = "mapreduce.job.id";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7e6f384d/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskProgressReporter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskProgressReporter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskProgressReporter.java
new file mode 100644
index 0000000..0bceb87
--- /dev/null
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestTaskProgressReporter.java
@@ -0,0 +1,160 @@
+/**
+* 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.mapred;
+
+import java.io.IOException;
+
+import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.mapred.SortedRanges.Range;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestTaskProgressReporter {
+  private static int statusUpdateTimes = 0;
+  private FakeUmbilical fakeUmbilical = new FakeUmbilical();
+
+  private static class DummyTask extends Task {
+    @Override
+    public void run(JobConf job, TaskUmbilicalProtocol umbilical)
+        throws IOException, ClassNotFoundException, InterruptedException {
+    }
+
+    @Override
+    public boolean isMapTask() {
+      return true;
+    }
+  }
+
+  private static class FakeUmbilical implements TaskUmbilicalProtocol {
+    @Override
+    public long getProtocolVersion(String protocol, long clientVersion)
+        throws IOException {
+      return 0;
+    }
+
+    @Override
+    public ProtocolSignature getProtocolSignature(String protocol,
+        long clientVersion, int clientMethodsHash) throws IOException {
+      return null;
+    }
+
+    @Override
+    public JvmTask getTask(JvmContext context) throws IOException {
+      return null;
+    }
+
+    @Override
+    public AMFeedback statusUpdate(TaskAttemptID taskId,
+        TaskStatus taskStatus) throws IOException, InterruptedException {
+      statusUpdateTimes++;
+      AMFeedback feedback = new AMFeedback();
+      feedback.setTaskFound(true);
+      feedback.setPreemption(true);
+      return feedback;
+    }
+
+    @Override
+    public void reportDiagnosticInfo(TaskAttemptID taskid, String trace)
+        throws IOException {
+    }
+
+    @Override
+    public void reportNextRecordRange(TaskAttemptID taskid, Range range)
+        throws IOException {
+    }
+
+    @Override
+    public void done(TaskAttemptID taskid) throws IOException {
+    }
+
+    @Override
+    public void commitPending(TaskAttemptID taskId, TaskStatus taskStatus)
+        throws IOException, InterruptedException {
+    }
+
+    @Override
+    public boolean canCommit(TaskAttemptID taskid) throws IOException {
+      return false;
+    }
+
+    @Override
+    public void shuffleError(TaskAttemptID taskId, String message)
+        throws IOException {
+    }
+
+    @Override
+    public void fsError(TaskAttemptID taskId, String message)
+        throws IOException {
+    }
+
+    @Override
+    public void fatalError(TaskAttemptID taskId, String message)
+        throws IOException {
+    }
+
+    @Override
+    public MapTaskCompletionEventsUpdate getMapCompletionEvents(
+        JobID jobId, int fromIndex, int maxLocs, TaskAttemptID id)
+        throws IOException {
+      return null;
+    }
+
+    @Override
+    public void preempted(TaskAttemptID taskId, TaskStatus taskStatus)
+        throws IOException, InterruptedException {
+    }
+
+    @Override
+    public TaskCheckpointID getCheckpointID(TaskID taskID) {
+      return null;
+    }
+
+    @Override
+    public void setCheckpointID(TaskID tid, TaskCheckpointID cid) {
+    }
+  }
+
+  private class DummyTaskReporter extends Task.TaskReporter {
+    public DummyTaskReporter(Task task) {
+      task.super(task.getProgress(), fakeUmbilical);
+    }
+    @Override
+    public void setProgress(float progress) {
+      super.setProgress(progress);
+    }
+  }
+
+  @Test (timeout=10000)
+  public void testTaskProgress() throws Exception {
+    JobConf job = new JobConf();
+    job.setLong(MRJobConfig.TASK_PROGRESS_REPORT_INTERVAL, 1000);
+    Task task = new DummyTask();
+    task.setConf(job);
+    DummyTaskReporter reporter = new DummyTaskReporter(task);
+    Thread t = new Thread(reporter);
+    t.start();
+    Thread.sleep(2100);
+    task.setTaskDone();
+    reporter.resetDoneFlag();
+    t.join();
+    Assert.assertEquals(statusUpdateTimes, 2);
+  }
+}
\ No newline at end of file


[25/51] [abbrv] hadoop git commit: HDFS-7917. Use file to replace data dirs in test to simulate a disk failure. Contributed by Lei (Eddy) Xu.

Posted by ka...@apache.org.
HDFS-7917. Use file to replace data dirs in test to simulate a disk failure. Contributed by Lei (Eddy) Xu.


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

Branch: refs/heads/YARN-2139
Commit: 2c238ae4e00371ef76582b007bb0e20ac8455d9c
Parents: 972f1f1
Author: cnauroth <cn...@apache.org>
Authored: Mon Mar 23 16:29:51 2015 -0700
Committer: cnauroth <cn...@apache.org>
Committed: Mon Mar 23 16:29:51 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hdfs/server/datanode/DataNodeTestUtils.java | 61 +++++++++++++++++++-
 .../datanode/TestDataNodeHotSwapVolumes.java    | 29 ++++------
 .../datanode/TestDataNodeVolumeFailure.java     | 11 +---
 .../TestDataNodeVolumeFailureReporting.java     | 46 ++++-----------
 .../TestDataNodeVolumeFailureToleration.java    |  8 +--
 6 files changed, 88 insertions(+), 70 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c238ae4/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 8c99876..b88b7e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -774,6 +774,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7962. Remove duplicated logs in BlockManager. (yliu)
 
+    HDFS-7917. Use file to replace data dirs in test to simulate a disk failure.
+    (Lei (Eddy) Xu via cnauroth)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c238ae4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
index fd51e52..f9a2ba1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/DataNodeTestUtils.java
@@ -40,7 +40,9 @@ import com.google.common.base.Preconditions;
  * Utility class for accessing package-private DataNode information during tests.
  *
  */
-public class DataNodeTestUtils {  
+public class DataNodeTestUtils {
+  private static final String DIR_FAILURE_SUFFIX = ".origin";
+
   public static DatanodeRegistration 
   getDNRegistrationForBP(DataNode dn, String bpid) throws IOException {
     return dn.getDNRegistrationForBP(bpid);
@@ -159,4 +161,61 @@ public class DataNodeTestUtils {
       final String bpid, final long blkId) {
     return FsDatasetTestUtil.fetchReplicaInfo(dn.getFSDataset(), bpid, blkId);
   }
+
+  /**
+   * It injects disk failures to data dirs by replacing these data dirs with
+   * regular files.
+   *
+   * @param dirs data directories.
+   * @throws IOException on I/O error.
+   */
+  public static void injectDataDirFailure(File... dirs) throws IOException {
+    for (File dir : dirs) {
+      File renamedTo = new File(dir.getPath() + DIR_FAILURE_SUFFIX);
+      if (renamedTo.exists()) {
+        throw new IOException(String.format(
+            "Can not inject failure to dir: %s because %s exists.",
+            dir, renamedTo));
+      }
+      if (!dir.renameTo(renamedTo)) {
+        throw new IOException(String.format("Failed to rename %s to %s.",
+            dir, renamedTo));
+      }
+      if (!dir.createNewFile()) {
+        throw new IOException(String.format(
+            "Failed to create file %s to inject disk failure.", dir));
+      }
+    }
+  }
+
+  /**
+   * Restore the injected data dir failures.
+   *
+   * @see {@link #injectDataDirFailures}.
+   * @param dirs data directories.
+   * @throws IOException
+   */
+  public static void restoreDataDirFromFailure(File... dirs)
+      throws IOException {
+    for (File dir : dirs) {
+      File renamedDir = new File(dir.getPath() + DIR_FAILURE_SUFFIX);
+      if (renamedDir.exists()) {
+        if (dir.exists()) {
+          if (!dir.isFile()) {
+            throw new IOException(
+                "Injected failure data dir is supposed to be file: " + dir);
+          }
+          if (!dir.delete()) {
+            throw new IOException(
+                "Failed to delete injected failure data dir: " + dir);
+          }
+        }
+        if (!renamedDir.renameTo(dir)) {
+          throw new IOException(String.format(
+              "Failed to recover injected failure data dir %s to %s.",
+              renamedDir, dir));
+        }
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c238ae4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
index 8ab3dd2..2f51d45 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeHotSwapVolumes.java
@@ -26,7 +26,6 @@ import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.BlockMissingException;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
@@ -682,26 +681,18 @@ public class TestDataNodeHotSwapVolumes {
         failedVolume != null);
     long used = failedVolume.getDfsUsed();
 
-    try {
-      assertTrue("Couldn't chmod local vol: " + dirToFail,
-          FileUtil.setExecutable(dirToFail, false));
-      // Call and wait DataNode to detect disk failure.
-      long lastDiskErrorCheck = dn.getLastDiskErrorCheck();
-      dn.checkDiskErrorAsync();
-      while (dn.getLastDiskErrorCheck() == lastDiskErrorCheck) {
-        Thread.sleep(100);
-      }
-
-      createFile(new Path("/test1"), 32, (short)2);
-      assertEquals(used, failedVolume.getDfsUsed());
-    } finally {
-      // Need to restore the mode on dirToFail. Otherwise, if an Exception
-      // is thrown above, the following tests can not delete this data directory
-      // and thus fail to start MiniDFSCluster.
-      assertTrue("Couldn't restore executable for: " + dirToFail,
-          FileUtil.setExecutable(dirToFail, true));
+    DataNodeTestUtils.injectDataDirFailure(dirToFail);
+    // Call and wait DataNode to detect disk failure.
+    long lastDiskErrorCheck = dn.getLastDiskErrorCheck();
+    dn.checkDiskErrorAsync();
+    while (dn.getLastDiskErrorCheck() == lastDiskErrorCheck) {
+      Thread.sleep(100);
     }
 
+    createFile(new Path("/test1"), 32, (short)2);
+    assertEquals(used, failedVolume.getDfsUsed());
+
+    DataNodeTestUtils.restoreDataDirFromFailure(dirToFail);
     dn.reconfigurePropertyImpl(DFS_DATANODE_DATA_DIR_KEY, oldDataDir);
 
     createFile(new Path("/test2"), 32, (short)2);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c238ae4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
index 9cbad6d..0428b81 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailure.java
@@ -121,10 +121,6 @@ public class TestDataNodeVolumeFailure {
     if(cluster != null) {
       cluster.shutdown();
     }
-    for (int i = 0; i < 3; i++) {
-      FileUtil.setExecutable(new File(dataDir, "data"+(2*i+1)), true);
-      FileUtil.setExecutable(new File(dataDir, "data"+(2*i+2)), true);
-    }
   }
   
   /*
@@ -159,7 +155,7 @@ public class TestDataNodeVolumeFailure {
         !deteteBlocks(failedDir)
         ) {
       throw new IOException("Could not delete hdfs directory '" + failedDir + "'");
-    }    
+    }
     data_fail.setReadOnly();
     failedDir.setReadOnly();
     System.out.println("Deleteing " + failedDir.getPath() + "; exist=" + failedDir.exists());
@@ -217,7 +213,7 @@ public class TestDataNodeVolumeFailure {
     DFSTestUtil.waitReplication(fs, file1, (short) 2);
 
     File dn0Vol1 = new File(dataDir, "data" + (2 * 0 + 1));
-    assertTrue(FileUtil.setExecutable(dn0Vol1, false));
+    DataNodeTestUtils.injectDataDirFailure(dn0Vol1);
     DataNode dn0 = cluster.getDataNodes().get(0);
     long lastDiskErrorCheck = dn0.getLastDiskErrorCheck();
     dn0.checkDiskErrorAsync();
@@ -291,8 +287,7 @@ public class TestDataNodeVolumeFailure {
     // Fail the first volume on both datanodes
     File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
     File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
+    DataNodeTestUtils.injectDataDirFailure(dn1Vol1, dn2Vol1);
 
     Path file2 = new Path("/test2");
     DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c238ae4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
index 9842f25..aac288a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureReporting.java
@@ -34,7 +34,6 @@ import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.ReconfigurationException;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
@@ -87,19 +86,6 @@ public class TestDataNodeVolumeFailureReporting {
 
   @After
   public void tearDown() throws Exception {
-    // Restore executable permission on all directories where a failure may have
-    // been simulated by denying execute access.  This is based on the maximum
-    // number of datanodes and the maximum number of storages per data node used
-    // throughout the tests in this suite.
-    assumeTrue(!Path.WINDOWS);
-    int maxDataNodes = 3;
-    int maxStoragesPerDataNode = 4;
-    for (int i = 0; i < maxDataNodes; i++) {
-      for (int j = 1; j <= maxStoragesPerDataNode; j++) {
-        String subDir = "data" + ((i * maxStoragesPerDataNode) + j);
-        FileUtil.setExecutable(new File(dataDir, subDir), true);
-      }
-    }
     IOUtils.cleanup(LOG, fs);
     if (cluster != null) {
       cluster.shutdown();
@@ -141,8 +127,7 @@ public class TestDataNodeVolumeFailureReporting {
      * fail. The client does not retry failed nodes even though
      * perhaps they could succeed because just a single volume failed.
      */
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
+    DataNodeTestUtils.injectDataDirFailure(dn1Vol1, dn2Vol1);
 
     /*
      * Create file1 and wait for 3 replicas (ie all DNs can still
@@ -179,7 +164,7 @@ public class TestDataNodeVolumeFailureReporting {
      * Now fail a volume on the third datanode. We should be able to get
      * three replicas since we've already identified the other failures.
      */
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol1, false));
+    DataNodeTestUtils.injectDataDirFailure(dn3Vol1);
     Path file2 = new Path("/test2");
     DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);
     DFSTestUtil.waitReplication(fs, file2, (short)3);
@@ -208,7 +193,7 @@ public class TestDataNodeVolumeFailureReporting {
      * and that it's no longer up. Only wait for two replicas since
      * we'll never get a third.
      */
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol2, false));
+    DataNodeTestUtils.injectDataDirFailure(dn3Vol2);
     Path file3 = new Path("/test3");
     DFSTestUtil.createFile(fs, file3, 1024, (short)3, 1L);
     DFSTestUtil.waitReplication(fs, file3, (short)2);
@@ -233,10 +218,8 @@ public class TestDataNodeVolumeFailureReporting {
      * restart, so file creation should be able to succeed after
      * restoring the data directories and restarting the datanodes.
      */
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, true));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, true));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol1, true));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn3Vol2, true));
+    DataNodeTestUtils.restoreDataDirFromFailure(
+        dn1Vol1, dn2Vol1, dn3Vol1, dn3Vol2);
     cluster.restartDataNodes();
     cluster.waitActive();
     Path file4 = new Path("/test4");
@@ -275,8 +258,7 @@ public class TestDataNodeVolumeFailureReporting {
     // third healthy so one node in the pipeline will not fail). 
     File dn1Vol1 = new File(dataDir, "data"+(2*0+1));
     File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
+    DataNodeTestUtils.injectDataDirFailure(dn1Vol1, dn2Vol1);
 
     Path file1 = new Path("/test1");
     DFSTestUtil.createFile(fs, file1, 1024, (short)2, 1L);
@@ -323,14 +305,7 @@ public class TestDataNodeVolumeFailureReporting {
 
     // Make the first two volume directories on the first two datanodes
     // non-accessible.
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1,
-        false));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol2,
-        false));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1,
-        false));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol2,
-        false));
+    DataNodeTestUtils.injectDataDirFailure(dn1Vol1, dn1Vol2, dn2Vol1, dn2Vol2);
 
     // Create file1 and wait for 3 replicas (ie all DNs can still store a block).
     // Then assert that all DNs are up, despite the volume failures.
@@ -380,8 +355,8 @@ public class TestDataNodeVolumeFailureReporting {
     File dn1Vol2 = new File(dataDir, "data"+(2*0+2));
     File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
     File dn2Vol2 = new File(dataDir, "data"+(2*1+2));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, false));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
+    DataNodeTestUtils.injectDataDirFailure(dn1Vol1);
+    DataNodeTestUtils.injectDataDirFailure(dn2Vol1);
 
     Path file1 = new Path("/test1");
     DFSTestUtil.createFile(fs, file1, 1024, (short)2, 1L);
@@ -449,8 +424,7 @@ public class TestDataNodeVolumeFailureReporting {
 
     // Replace failed volume with healthy volume and run reconfigure DataNode.
     // The failed volume information should be cleared.
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn1Vol1, true));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, true));
+    DataNodeTestUtils.restoreDataDirFromFailure(dn1Vol1, dn2Vol1);
     reconfigureDataNode(dns.get(0), dn1Vol1, dn1Vol2);
     reconfigureDataNode(dns.get(1), dn2Vol1, dn2Vol2);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c238ae4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
index 73dc77c..5b7ac30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeVolumeFailureToleration.java
@@ -76,10 +76,6 @@ public class TestDataNodeVolumeFailureToleration {
 
   @After
   public void tearDown() throws Exception {
-    for (int i = 0; i < 3; i++) {
-      FileUtil.setExecutable(new File(dataDir, "data"+(2*i+1)), true);
-      FileUtil.setExecutable(new File(dataDir, "data"+(2*i+2)), true);
-    }
     cluster.shutdown();
   }
 
@@ -152,7 +148,7 @@ public class TestDataNodeVolumeFailureToleration {
 
     // Fail a volume on the 2nd DN
     File dn2Vol1 = new File(dataDir, "data"+(2*1+1));
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, false));
+    DataNodeTestUtils.injectDataDirFailure(dn2Vol1);
 
     // Should only get two replicas (the first DN and the 3rd)
     Path file1 = new Path("/test1");
@@ -165,7 +161,7 @@ public class TestDataNodeVolumeFailureToleration {
 
     // If we restore the volume we should still only be able to get
     // two replicas since the DN is still considered dead.
-    assertTrue("Couldn't chmod local vol", FileUtil.setExecutable(dn2Vol1, true));
+    DataNodeTestUtils.restoreDataDirFromFailure(dn2Vol1);
     Path file2 = new Path("/test2");
     DFSTestUtil.createFile(fs, file2, 1024, (short)3, 1L);
     DFSTestUtil.waitReplication(fs, file2, (short)2);


[13/51] [abbrv] hadoop git commit: MAPREDUCE-5448. MapFileOutputFormat#getReaders bug with invisible files/folders. Contributed by Maysam Yabandeh.

Posted by ka...@apache.org.
MAPREDUCE-5448. MapFileOutputFormat#getReaders bug with invisible files/folders. Contributed by Maysam Yabandeh.


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

Branch: refs/heads/YARN-2139
Commit: b46c2bb51ae524e6640756620f70e5925cda7592
Parents: 4335429
Author: Harsh J <ha...@cloudera.com>
Authored: Sun Mar 22 09:45:48 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Sun Mar 22 09:45:48 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                    |  3 +++
 .../mapreduce/lib/output/MapFileOutputFormat.java       | 12 +++++++++++-
 .../mapreduce/lib/output/TestFileOutputCommitter.java   | 10 ++++++++++
 3 files changed, 24 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b46c2bb5/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index fc42941..2920811 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -286,6 +286,9 @@ Release 2.8.0 - UNRELEASED
 
   BUG FIXES
 
+    MAPREDUCE-5448. MapFileOutputFormat#getReaders bug with hidden
+    files/folders. (Maysam Yabandeh via harsh)
+
     MAPREDUCE-6286. A typo in HistoryViewer makes some code useless, which
     causes counter limits are not reset correctly.
     (Zhihai Xu via harsh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b46c2bb5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MapFileOutputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MapFileOutputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MapFileOutputFormat.java
index b8cb997..da33770 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MapFileOutputFormat.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MapFileOutputFormat.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.PathFilter;
 
 import org.apache.hadoop.io.MapFile;
 import org.apache.hadoop.io.WritableComparable;
@@ -88,7 +89,16 @@ public class MapFileOutputFormat
   public static MapFile.Reader[] getReaders(Path dir,
       Configuration conf) throws IOException {
     FileSystem fs = dir.getFileSystem(conf);
-    Path[] names = FileUtil.stat2Paths(fs.listStatus(dir));
+    PathFilter filter = new PathFilter() {
+      @Override
+      public boolean accept(Path path) {
+        String name = path.getName();
+        if (name.startsWith("_") || name.startsWith("."))
+          return false;
+        return true;
+      }
+    };
+    Path[] names = FileUtil.stat2Paths(fs.listStatus(dir, filter));
 
     // sort names, so that hash partitioning works
     Arrays.sort(names);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b46c2bb5/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java
index 0d4ab98..5c4428b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/output/TestFileOutputCommitter.java
@@ -27,6 +27,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
+import junit.framework.Assert;
 import junit.framework.TestCase;
 
 import org.apache.commons.logging.Log;
@@ -309,6 +310,15 @@ public class TestFileOutputCommitter extends TestCase {
     committer.commitTask(tContext);
     committer.commitJob(jContext);
 
+    // Ensure getReaders call works and also ignores
+    // hidden filenames (_ or . prefixes)
+    try {
+      MapFileOutputFormat.getReaders(outDir, conf);
+    } catch (Exception e) {
+      Assert.fail("Fail to read from MapFileOutputFormat: " + e);
+      e.printStackTrace();
+    }
+
     // validate output
     validateMapFileOutputContent(FileSystem.get(job.getConfiguration()), outDir);
     FileUtil.fullyDelete(new File(outDir.toString()));


[34/51] [abbrv] hadoop git commit: MAPREDUCE-6285. ClientServiceDelegate should not retry upon AuthenticationException. Contributed by Jonathan Eagles.

Posted by ka...@apache.org.
MAPREDUCE-6285. ClientServiceDelegate should not retry upon AuthenticationException. Contributed by Jonathan Eagles.


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

Branch: refs/heads/YARN-2139
Commit: 4170c99147b0cb6d561ff626cea140e0a061b314
Parents: 3ca5bd1
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed Mar 25 00:56:26 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Wed Mar 25 00:56:26 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |  3 ++
 .../hadoop/mapred/ClientServiceDelegate.java    |  6 +++
 .../mapred/TestClientServiceDelegate.java       | 44 ++++++++++++++++++++
 3 files changed, 53 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/4170c991/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index b8a2a1c..2b16c30 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -496,6 +496,9 @@ Release 2.7.0 - UNRELEASED
     MAPREDUCE-6275. Race condition in FileOutputCommitter v2 for
     user-specified task output subdirs (Gera Shegalov and Siqi Li via jlowe)
 
+    MAPREDUCE-6285. ClientServiceDelegate should not retry upon
+    AuthenticationException. (Jonathan Eagles via ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4170c991/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
index 686fa0c..8517c19 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptReport;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -328,6 +329,11 @@ public class ClientServiceDelegate {
         // Force reconnection by setting the proxy to null.
         realProxy = null;
         // HS/AMS shut down
+
+        if (e.getCause() instanceof AuthorizationException) {
+          throw new IOException(e.getTargetException());
+        }
+
         // if it's AM shut down, do not decrement maxClientRetry as we wait for
         // AM to be restarted.
         if (!usingAMProxy.get()) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/4170c991/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
index 7d6b2f3..b85f18d 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientServiceDelegate.java
@@ -48,6 +48,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.Counters;
 import org.apache.hadoop.mapreduce.v2.api.records.JobReport;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+import org.apache.hadoop.security.authorize.AuthorizationException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -183,6 +184,49 @@ public class TestClientServiceDelegate {
   }
 
   @Test
+  public void testNoRetryOnAMAuthorizationException() throws Exception {
+    if (!isAMReachableFromClient) {
+      return;
+    }
+
+    ResourceMgrDelegate rm = mock(ResourceMgrDelegate.class);
+    when(rm.getApplicationReport(TypeConverter.toYarn(oldJobId).getAppId()))
+      .thenReturn(getRunningApplicationReport("am1", 78));
+
+    // throw authorization exception on first invocation
+    final MRClientProtocol amProxy = mock(MRClientProtocol.class);
+    when(amProxy.getJobReport(any(GetJobReportRequest.class)))
+      .thenThrow(new AuthorizationException("Denied"));
+    Configuration conf = new YarnConfiguration();
+    conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME);
+    conf.setBoolean(MRJobConfig.JOB_AM_ACCESS_DISABLED,
+      !isAMReachableFromClient);
+    ClientServiceDelegate clientServiceDelegate =
+        new ClientServiceDelegate(conf, rm, oldJobId, null) {
+          @Override
+          MRClientProtocol instantiateAMProxy(
+              final InetSocketAddress serviceAddr) throws IOException {
+            super.instantiateAMProxy(serviceAddr);
+            return amProxy;
+          }
+        };
+
+    try {
+      clientServiceDelegate.getJobStatus(oldJobId);
+      Assert.fail("Exception should be thrown upon AuthorizationException");
+    } catch (IOException e) {
+      Assert.assertEquals(AuthorizationException.class.getName() + ": Denied",
+          e.getMessage());
+    }
+
+    // assert maxClientRetry is not decremented.
+    Assert.assertEquals(conf.getInt(MRJobConfig.MR_CLIENT_MAX_RETRIES,
+      MRJobConfig.DEFAULT_MR_CLIENT_MAX_RETRIES), clientServiceDelegate
+      .getMaxClientRetry());
+    verify(amProxy, times(1)).getJobReport(any(GetJobReportRequest.class));
+  }
+
+  @Test
   public void testHistoryServerNotConfigured() throws Exception {
     //RM doesn't have app report and job History Server is not configured
     ClientServiceDelegate clientServiceDelegate = getClientServiceDelegate(


[24/51] [abbrv] hadoop git commit: YARN-2868. FairScheduler: Metric for latency to allocate first container for an application. (Ray Chiang via kasha)

Posted by ka...@apache.org.
YARN-2868. FairScheduler: Metric for latency to allocate first container for an application. (Ray Chiang via kasha)


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

Branch: refs/heads/YARN-2139
Commit: 972f1f1ab94a26ec446a272ad030fe13f03ed442
Parents: 2bc097c
Author: Karthik Kambatla <ka...@apache.org>
Authored: Mon Mar 23 14:07:05 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Mon Mar 23 14:07:05 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                    |  3 +++
 .../resourcemanager/scheduler/QueueMetrics.java    |  8 +++++++-
 .../scheduler/SchedulerApplicationAttempt.java     | 17 +++++++++++++++++
 .../scheduler/fair/FairScheduler.java              | 11 ++++++++++-
 4 files changed, 37 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/972f1f1a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index b716064..e7d4f59 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -73,6 +73,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3350. YARN RackResolver spams logs with messages at info level. 
     (Wilfred Spiegelenburg via junping_du)
 
+    YARN-2868. FairScheduler: Metric for latency to allocate first container 
+    for an application. (Ray Chiang via kasha)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/972f1f1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.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/QueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
index 507b798..58b1ed1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java
@@ -38,6 +38,7 @@ import org.apache.hadoop.metrics2.lib.MetricsRegistry;
 import org.apache.hadoop.metrics2.lib.MutableCounterInt;
 import org.apache.hadoop.metrics2.lib.MutableCounterLong;
 import org.apache.hadoop.metrics2.lib.MutableGaugeInt;
+import org.apache.hadoop.metrics2.lib.MutableRate;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -74,6 +75,7 @@ public class QueueMetrics implements MetricsSource {
   @Metric("# of reserved containers") MutableGaugeInt reservedContainers;
   @Metric("# of active users") MutableGaugeInt activeUsers;
   @Metric("# of active applications") MutableGaugeInt activeApplications;
+  @Metric("App Attempt First Container Allocation Delay") MutableRate appAttemptFirstContainerAllocationDelay;
   private final MutableGaugeInt[] runningTime;
   private TimeBucketMetrics<ApplicationId> runBuckets;
 
@@ -462,7 +464,11 @@ public class QueueMetrics implements MetricsSource {
       parent.deactivateApp(user);
     }
   }
-  
+
+  public void addAppAttemptFirstContainerAllocationDelay(long latency) {
+    appAttemptFirstContainerAllocationDelay.add(latency);
+  }
+
   public int getAppsSubmitted() {
     return appsSubmitted.value();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/972f1f1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.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/SchedulerApplicationAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
index 799a5c1..bf5641d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/SchedulerApplicationAttempt.java
@@ -25,6 +25,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.commons.lang.time.DateUtils;
 import org.apache.commons.logging.Log;
@@ -93,6 +94,8 @@ public class SchedulerApplicationAttempt {
   private LogAggregationContext logAggregationContext;
   
   protected ResourceUsage attemptResourceUsage = new ResourceUsage();
+  private AtomicLong firstAllocationRequestSentTime = new AtomicLong(0);
+  private AtomicLong firstContainerAllocatedTime = new AtomicLong(0);
 
   protected List<RMContainer> newlyAllocatedContainers = 
       new ArrayList<RMContainer>();
@@ -648,4 +651,18 @@ public class SchedulerApplicationAttempt {
           Resources.clone(headroom));
     }
   }
+
+  public void recordContainerRequestTime(long value) {
+    firstAllocationRequestSentTime.compareAndSet(0, value);
+  }
+
+  public void recordContainerAllocationTime(long value) {
+    if (firstContainerAllocatedTime.compareAndSet(0, value)) {
+      long timediff = firstContainerAllocatedTime.longValue() -
+          firstAllocationRequestSentTime.longValue();
+      if (timediff > 0) {
+        queue.getMetrics().addAppAttemptFirstContainerAllocationDelay(timediff);
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/972f1f1a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.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/fair/FairScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
index 98a8de2..04c7f70 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/FairScheduler.java
@@ -898,6 +898,9 @@ public class FairScheduler extends
         clusterResource, minimumAllocation, getMaximumResourceCapability(),
         incrAllocation);
 
+    // Record container allocation start time
+    application.recordContainerRequestTime(getClock().getTime());
+
     // Set amResource for this app
     if (!application.getUnmanagedAM() && ask.size() == 1
         && application.getLiveContainers().isEmpty()) {
@@ -931,7 +934,7 @@ public class FairScheduler extends
         LOG.debug("Preempting " + application.getPreemptionContainers().size()
             + " container(s)");
       }
-      
+
       Set<ContainerId> preemptionContainerIds = new HashSet<ContainerId>();
       for (RMContainer container : application.getPreemptionContainers()) {
         preemptionContainerIds.add(container.getContainerId());
@@ -940,6 +943,12 @@ public class FairScheduler extends
       application.updateBlacklist(blacklistAdditions, blacklistRemovals);
       ContainersAndNMTokensAllocation allocation =
           application.pullNewlyAllocatedContainersAndNMTokens();
+
+      // Record container allocation time
+      if (!(allocation.getContainerList().isEmpty())) {
+        application.recordContainerAllocationTime(getClock().getTime());
+      }
+
       Resource headroom = application.getHeadroom();
       application.setApplicationHeadroomForMetrics(headroom);
       return new Allocation(allocation.getContainerList(), headroom,


[38/51] [abbrv] hadoop git commit: YARN-3383. AdminService should use warn instead of info to log exception when operation fails. (Li Lu via wangda)

Posted by ka...@apache.org.
YARN-3383. AdminService should use warn instead of info to log exception when operation fails. (Li Lu via wangda)


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

Branch: refs/heads/YARN-2139
Commit: 97a7277a2d696474b5c8e2d814c8291d4bde246e
Parents: eda0254
Author: Wangda Tan <wa...@apache.org>
Authored: Tue Mar 24 10:33:09 2015 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Tue Mar 24 10:38:54 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/97a7277a/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 8a5e142..6a495d9 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -103,6 +103,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3241. FairScheduler handles "invalid" queue names inconsistently. 
     (Zhihai Xu via kasha)
 
+    YARN-3383. AdminService should use "warn" instead of "info" to log exception 
+    when operation fails. (Li Lu via wangda)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/97a7277a/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 22b92c2..12714de 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
@@ -699,7 +699,7 @@ public class AdminService extends CompositeService implements
 
   private YarnException logAndWrapException(Exception exception, String user,
       String argName, String msg) throws YarnException {
-    LOG.info("Exception " + msg, exception);
+    LOG.warn("Exception " + msg, exception);
     RMAuditLogger.logFailure(user, argName, "", 
         "AdminService", "Exception " + msg);
     return RPCUtil.getRemoteException(exception);


[48/51] [abbrv] hadoop git commit: MAPREDUCE-579. Streaming slowmatch documentation.

Posted by ka...@apache.org.
MAPREDUCE-579. Streaming slowmatch documentation.


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

Branch: refs/heads/YARN-2139
Commit: b6dea9776b92c46d2ca593f7ada0a3b5dfdc2e04
Parents: 323945b
Author: Harsh J <ha...@cloudera.com>
Authored: Wed Mar 25 14:38:12 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Wed Mar 25 14:39:00 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                          | 2 ++
 .../hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm  | 7 +++++++
 2 files changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6dea977/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 2b16c30..f81a13f 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -256,6 +256,8 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-579. Streaming "slowmatch" documentation. (harsh)
+
     MAPREDUCE-6287. Deprecated methods in org.apache.hadoop.examples.Sort
     (Chao Zhang via harsh)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6dea977/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
index b4c5e38..7f2412e 100644
--- a/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
+++ b/hadoop-tools/hadoop-streaming/src/site/markdown/HadoopStreaming.md.vm
@@ -546,6 +546,13 @@ You can use the record reader StreamXmlRecordReader to process XML documents.
 
 Anything found between BEGIN\_STRING and END\_STRING would be treated as one record for map tasks.
 
+The name-value properties that StreamXmlRecordReader understands are:
+
+*   (strings) 'begin' - Characters marking beginning of record, and 'end' - Characters marking end of record.
+*   (boolean) 'slowmatch' - Toggle to look for begin and end characters, but within CDATA instead of regular tags. Defaults to false.
+*   (integer) 'lookahead' - Maximum lookahead bytes to sync CDATA when using 'slowmatch', should be larger than 'maxrec'. Defaults to 2*'maxrec'.
+*   (integer) 'maxrec' - Maximum record size to read between each match during 'slowmatch'. Defaults to 50000 bytes.
+
 $H3 How do I update counters in streaming applications?
 
 A streaming process can use the stderr to emit counter information. `reporter:counter:<group>,<counter>,<amount>` should be sent to stderr to update the counter.


[10/51] [abbrv] hadoop git commit: MAPREDUCE-6239. Consolidate TestJobConf classes in hadoop-mapreduce-client-jobclient and hadoop-mapreduce-client-core. Contributed by Varun Saxena.

Posted by ka...@apache.org.
MAPREDUCE-6239. Consolidate TestJobConf classes in hadoop-mapreduce-client-jobclient and hadoop-mapreduce-client-core. 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/7a678db3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/7a678db3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/7a678db3

Branch: refs/heads/YARN-2139
Commit: 7a678db3accf9480f3799dcf6fd7ffef09a311cc
Parents: e1feb4e
Author: Harsh J <ha...@cloudera.com>
Authored: Sat Mar 21 09:43:29 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Sat Mar 21 09:43:29 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt            |   4 +
 .../org/apache/hadoop/mapred/TestJobConf.java   | 173 ++++++++++++++++
 .../org/apache/hadoop/conf/TestJobConf.java     | 199 -------------------
 3 files changed, 177 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a678db3/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 48eda8b..4f80411 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -256,6 +256,10 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-6239. Consolidate TestJobConf classes in
+    hadoop-mapreduce-client-jobclient and hadoop-mapreduce-client-core
+    (Varun Saxena via harsh)
+
     MAPREDUCE-5807. Print usage by TeraSort job. (Rohith via harsh)
 
     MAPREDUCE-4653. TestRandomAlgorithm has an unused "import" statement.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a678db3/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
index 3d924e1..0612ade 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapred/TestJobConf.java
@@ -22,6 +22,7 @@ import java.util.regex.Pattern;
 import static org.junit.Assert.*;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -188,4 +189,176 @@ public class TestJobConf {
     Assert.assertEquals(2048, configuration.getLong(
         JobConf.MAPREDUCE_JOB_REDUCE_MEMORY_MB_PROPERTY, -1));
   }
+
+
+  @Test
+  public void testProfileParamsDefaults() {
+    JobConf configuration = new JobConf();
+    String result = configuration.getProfileParams();
+    Assert.assertNotNull(result);
+    Assert.assertTrue(result.contains("file=%s"));
+    Assert.assertTrue(result.startsWith("-agentlib:hprof"));
+  }
+
+  @Test
+  public void testProfileParamsSetter() {
+    JobConf configuration = new JobConf();
+
+    configuration.setProfileParams("test");
+    Assert.assertEquals("test", configuration.get(MRJobConfig.TASK_PROFILE_PARAMS));
+  }
+
+  @Test
+  public void testProfileParamsGetter() {
+    JobConf configuration = new JobConf();
+
+    configuration.set(MRJobConfig.TASK_PROFILE_PARAMS, "test");
+    Assert.assertEquals("test", configuration.getProfileParams());
+  }
+
+  /**
+   * Testing mapred.task.maxvmem replacement with new values
+   *
+   */
+  @Test
+  public void testMemoryConfigForMapOrReduceTask(){
+    JobConf configuration = new JobConf();
+    configuration.set(MRJobConfig.MAP_MEMORY_MB,String.valueOf(300));
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB,String.valueOf(300));
+    Assert.assertEquals(configuration.getMemoryForMapTask(),300);
+    Assert.assertEquals(configuration.getMemoryForReduceTask(),300);
+
+    configuration.set("mapred.task.maxvmem" , String.valueOf(2*1024 * 1024));
+    configuration.set(MRJobConfig.MAP_MEMORY_MB,String.valueOf(300));
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB,String.valueOf(300));
+    Assert.assertEquals(configuration.getMemoryForMapTask(),2);
+    Assert.assertEquals(configuration.getMemoryForReduceTask(),2);
+
+    configuration = new JobConf();
+    configuration.set("mapred.task.maxvmem" , "-1");
+    configuration.set(MRJobConfig.MAP_MEMORY_MB,String.valueOf(300));
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB,String.valueOf(400));
+    Assert.assertEquals(configuration.getMemoryForMapTask(), 300);
+    Assert.assertEquals(configuration.getMemoryForReduceTask(), 400);
+
+    configuration = new JobConf();
+    configuration.set("mapred.task.maxvmem" , String.valueOf(2*1024 * 1024));
+    configuration.set(MRJobConfig.MAP_MEMORY_MB,"-1");
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB,"-1");
+    Assert.assertEquals(configuration.getMemoryForMapTask(),2);
+    Assert.assertEquals(configuration.getMemoryForReduceTask(),2);
+
+    configuration = new JobConf();
+    configuration.set("mapred.task.maxvmem" , String.valueOf(-1));
+    configuration.set(MRJobConfig.MAP_MEMORY_MB,"-1");
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB,"-1");
+    Assert.assertEquals(configuration.getMemoryForMapTask(),
+        MRJobConfig.DEFAULT_MAP_MEMORY_MB);
+    Assert.assertEquals(configuration.getMemoryForReduceTask(),
+        MRJobConfig.DEFAULT_REDUCE_MEMORY_MB);
+
+    configuration = new JobConf();
+    configuration.set("mapred.task.maxvmem" , String.valueOf(2*1024 * 1024));
+    configuration.set(MRJobConfig.MAP_MEMORY_MB, "3");
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, "3");
+    Assert.assertEquals(configuration.getMemoryForMapTask(),2);
+    Assert.assertEquals(configuration.getMemoryForReduceTask(),2);
+  }
+
+  /**
+   * Test that negative values for MAPRED_TASK_MAXVMEM_PROPERTY cause
+   * new configuration keys' values to be used.
+   */
+  @Test
+  public void testNegativeValueForTaskVmem() {
+    JobConf configuration = new JobConf();
+
+    configuration.set(JobConf.MAPRED_TASK_MAXVMEM_PROPERTY, "-3");
+    Assert.assertEquals(MRJobConfig.DEFAULT_MAP_MEMORY_MB,
+        configuration.getMemoryForMapTask());
+    Assert.assertEquals(MRJobConfig.DEFAULT_REDUCE_MEMORY_MB,
+        configuration.getMemoryForReduceTask());
+
+    configuration.set(MRJobConfig.MAP_MEMORY_MB, "4");
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, "5");
+    Assert.assertEquals(4, configuration.getMemoryForMapTask());
+    Assert.assertEquals(5, configuration.getMemoryForReduceTask());
+
+  }
+
+  /**
+   * Test that negative values for new configuration keys get passed through.
+   */
+  @Test
+  public void testNegativeValuesForMemoryParams() {
+    JobConf configuration = new JobConf();
+
+    configuration.set(MRJobConfig.MAP_MEMORY_MB, "-5");
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, "-6");
+    Assert.assertEquals(MRJobConfig.DEFAULT_MAP_MEMORY_MB,
+        configuration.getMemoryForMapTask());
+    Assert.assertEquals(MRJobConfig.DEFAULT_REDUCE_MEMORY_MB,
+        configuration.getMemoryForReduceTask());
+  }
+
+  /**
+   *   Test deprecated accessor and mutator method for mapred.task.maxvmem
+   */
+  @Test
+  public void testMaxVirtualMemoryForTask() {
+    JobConf configuration = new JobConf();
+
+    //get test case
+    configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(300));
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, String.valueOf(-1));
+    Assert.assertEquals(
+      configuration.getMaxVirtualMemoryForTask(), 1024 * 1024 * 1024);
+
+    configuration = new JobConf();
+    configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(-1));
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, String.valueOf(200));
+    Assert.assertEquals(
+      configuration.getMaxVirtualMemoryForTask(), 1024 * 1024 * 1024);
+
+    configuration = new JobConf();
+    configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(-1));
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, String.valueOf(-1));
+    configuration.set("mapred.task.maxvmem", String.valueOf(1 * 1024 * 1024));
+    Assert.assertEquals(
+      configuration.getMaxVirtualMemoryForTask(), 1 * 1024 * 1024);
+
+    configuration = new JobConf();
+    configuration.set("mapred.task.maxvmem", String.valueOf(1 * 1024 * 1024));
+    Assert.assertEquals(
+      configuration.getMaxVirtualMemoryForTask(), 1 * 1024 * 1024);
+
+    //set test case
+
+    configuration = new JobConf();
+    configuration.setMaxVirtualMemoryForTask(2 * 1024 * 1024);
+    Assert.assertEquals(configuration.getMemoryForMapTask(), 2);
+    Assert.assertEquals(configuration.getMemoryForReduceTask(), 2);
+
+    configuration = new JobConf();
+    configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(300));
+    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, String.valueOf(400));
+    configuration.setMaxVirtualMemoryForTask(2 * 1024 * 1024);
+    Assert.assertEquals(configuration.getMemoryForMapTask(), 2);
+    Assert.assertEquals(configuration.getMemoryForReduceTask(), 2);
+  }
+
+  /**
+   * Ensure that by default JobContext.MAX_TASK_FAILURES_PER_TRACKER is less
+   * JobContext.MAP_MAX_ATTEMPTS and JobContext.REDUCE_MAX_ATTEMPTS so that
+   * failed tasks will be retried on other nodes
+   */
+  @Test
+  public void testMaxTaskFailuresPerTracker() {
+    JobConf jobConf = new JobConf(true);
+    Assert.assertTrue("By default JobContext.MAX_TASK_FAILURES_PER_TRACKER was "
+      + "not less than JobContext.MAP_MAX_ATTEMPTS and REDUCE_MAX_ATTEMPTS"
+      ,jobConf.getMaxTaskFailuresPerTracker() < jobConf.getMaxMapAttempts() &&
+      jobConf.getMaxTaskFailuresPerTracker() < jobConf.getMaxReduceAttempts()
+      );
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7a678db3/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestJobConf.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestJobConf.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestJobConf.java
deleted file mode 100644
index f67ba1f..0000000
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/conf/TestJobConf.java
+++ /dev/null
@@ -1,199 +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.conf;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-
-public class TestJobConf {
-
-  @Test
-  public void testProfileParamsDefaults() {
-    JobConf configuration = new JobConf();
-    String result = configuration.getProfileParams();
-    Assert.assertNotNull(result);
-    Assert.assertTrue(result.contains("file=%s"));
-    Assert.assertTrue(result.startsWith("-agentlib:hprof"));
-  }
-
-  @Test
-  public void testProfileParamsSetter() {
-    JobConf configuration = new JobConf();
-
-    configuration.setProfileParams("test");
-    Assert.assertEquals("test", configuration.get(MRJobConfig.TASK_PROFILE_PARAMS));
-  }
-
-  @Test
-  public void testProfileParamsGetter() {
-    JobConf configuration = new JobConf();
-
-    configuration.set(MRJobConfig.TASK_PROFILE_PARAMS, "test");
-    Assert.assertEquals("test", configuration.getProfileParams());
-  }
-
-  /**
-   * Testing mapred.task.maxvmem replacement with new values
-   *
-   */
-  @Test
-  public void testMemoryConfigForMapOrReduceTask(){
-    JobConf configuration = new JobConf();
-    configuration.set(MRJobConfig.MAP_MEMORY_MB,String.valueOf(300));
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB,String.valueOf(300));
-    Assert.assertEquals(configuration.getMemoryForMapTask(),300);
-    Assert.assertEquals(configuration.getMemoryForReduceTask(),300);
-
-    configuration.set("mapred.task.maxvmem" , String.valueOf(2*1024 * 1024));
-    configuration.set(MRJobConfig.MAP_MEMORY_MB,String.valueOf(300));
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB,String.valueOf(300));
-    Assert.assertEquals(configuration.getMemoryForMapTask(),2);
-    Assert.assertEquals(configuration.getMemoryForReduceTask(),2);
-
-    configuration = new JobConf();
-    configuration.set("mapred.task.maxvmem" , "-1");
-    configuration.set(MRJobConfig.MAP_MEMORY_MB,String.valueOf(300));
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB,String.valueOf(400));
-    Assert.assertEquals(configuration.getMemoryForMapTask(), 300);
-    Assert.assertEquals(configuration.getMemoryForReduceTask(), 400);
-
-    configuration = new JobConf();
-    configuration.set("mapred.task.maxvmem" , String.valueOf(2*1024 * 1024));
-    configuration.set(MRJobConfig.MAP_MEMORY_MB,"-1");
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB,"-1");
-    Assert.assertEquals(configuration.getMemoryForMapTask(),2);
-    Assert.assertEquals(configuration.getMemoryForReduceTask(),2);
-
-    configuration = new JobConf();
-    configuration.set("mapred.task.maxvmem" , String.valueOf(-1));
-    configuration.set(MRJobConfig.MAP_MEMORY_MB,"-1");
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB,"-1");
-    Assert.assertEquals(configuration.getMemoryForMapTask(),
-        MRJobConfig.DEFAULT_MAP_MEMORY_MB);
-    Assert.assertEquals(configuration.getMemoryForReduceTask(),
-        MRJobConfig.DEFAULT_REDUCE_MEMORY_MB);
-
-    configuration = new JobConf();
-    configuration.set("mapred.task.maxvmem" , String.valueOf(2*1024 * 1024));
-    configuration.set(MRJobConfig.MAP_MEMORY_MB, "3");
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, "3");
-    Assert.assertEquals(configuration.getMemoryForMapTask(),2);
-    Assert.assertEquals(configuration.getMemoryForReduceTask(),2);
-  }
-
-  /**
-   * Test that negative values for MAPRED_TASK_MAXVMEM_PROPERTY cause
-   * new configuration keys' values to be used.
-   */
-  @Test
-  public void testNegativeValueForTaskVmem() {
-    JobConf configuration = new JobConf();
-
-    configuration.set(JobConf.MAPRED_TASK_MAXVMEM_PROPERTY, "-3");
-    Assert.assertEquals(MRJobConfig.DEFAULT_MAP_MEMORY_MB,
-        configuration.getMemoryForMapTask());
-    Assert.assertEquals(MRJobConfig.DEFAULT_REDUCE_MEMORY_MB,
-        configuration.getMemoryForReduceTask());
-    
-    configuration.set(MRJobConfig.MAP_MEMORY_MB, "4");
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, "5");
-    Assert.assertEquals(4, configuration.getMemoryForMapTask());
-    Assert.assertEquals(5, configuration.getMemoryForReduceTask());
-    
-  }
-  
-  /**
-   * Test that negative values for new configuration keys get passed through.
-   */
-  @Test
-  public void testNegativeValuesForMemoryParams() {
-    JobConf configuration = new JobConf();
-        
-    configuration.set(MRJobConfig.MAP_MEMORY_MB, "-5");
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, "-6");
-    Assert.assertEquals(MRJobConfig.DEFAULT_MAP_MEMORY_MB,
-        configuration.getMemoryForMapTask());
-    Assert.assertEquals(MRJobConfig.DEFAULT_REDUCE_MEMORY_MB,
-        configuration.getMemoryForReduceTask());
-  }
-
-  /**
-   *   Test deprecated accessor and mutator method for mapred.task.maxvmem
-   */
-  @Test
-  public void testMaxVirtualMemoryForTask() {
-    JobConf configuration = new JobConf();
-
-    //get test case
-    configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(300));
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, String.valueOf(-1));
-    Assert.assertEquals(
-      configuration.getMaxVirtualMemoryForTask(), 1024 * 1024 * 1024);
-
-    configuration = new JobConf();
-    configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(-1));
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, String.valueOf(200));
-    Assert.assertEquals(
-      configuration.getMaxVirtualMemoryForTask(), 1024 * 1024 * 1024);
-
-    configuration = new JobConf();
-    configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(-1));
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, String.valueOf(-1));
-    configuration.set("mapred.task.maxvmem", String.valueOf(1 * 1024 * 1024));
-    Assert.assertEquals(
-      configuration.getMaxVirtualMemoryForTask(), 1 * 1024 * 1024);
-
-    configuration = new JobConf();
-    configuration.set("mapred.task.maxvmem", String.valueOf(1 * 1024 * 1024));
-    Assert.assertEquals(
-      configuration.getMaxVirtualMemoryForTask(), 1 * 1024 * 1024);
-
-    //set test case
-
-    configuration = new JobConf();
-    configuration.setMaxVirtualMemoryForTask(2 * 1024 * 1024);
-    Assert.assertEquals(configuration.getMemoryForMapTask(), 2);
-    Assert.assertEquals(configuration.getMemoryForReduceTask(), 2);
-
-    configuration = new JobConf();   
-    configuration.set(MRJobConfig.MAP_MEMORY_MB, String.valueOf(300));
-    configuration.set(MRJobConfig.REDUCE_MEMORY_MB, String.valueOf(400));
-    configuration.setMaxVirtualMemoryForTask(2 * 1024 * 1024);
-    Assert.assertEquals(configuration.getMemoryForMapTask(), 2);
-    Assert.assertEquals(configuration.getMemoryForReduceTask(), 2);
-    
-    
-  }
-
-  /**
-   * Ensure that by default JobContext.MAX_TASK_FAILURES_PER_TRACKER is less
-   * JobContext.MAP_MAX_ATTEMPTS and JobContext.REDUCE_MAX_ATTEMPTS so that
-   * failed tasks will be retried on other nodes
-   */
-  @Test
-  public void testMaxTaskFailuresPerTracker() {
-    JobConf jobConf = new JobConf(true);
-    Assert.assertTrue("By default JobContext.MAX_TASK_FAILURES_PER_TRACKER was "
-      + "not less than JobContext.MAP_MAX_ATTEMPTS and REDUCE_MAX_ATTEMPTS"
-      ,jobConf.getMaxTaskFailuresPerTracker() < jobConf.getMaxMapAttempts() &&
-      jobConf.getMaxTaskFailuresPerTracker() < jobConf.getMaxReduceAttempts()
-      );
-  }
-}


[51/51] [abbrv] hadoop git commit: YARN-3397. yarn rmadmin should skip -failover. (J.Andreina via kasha)

Posted by ka...@apache.org.
YARN-3397. yarn rmadmin should skip -failover. (J.Andreina via kasha)


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

Branch: refs/heads/YARN-2139
Commit: c906a1de7280dabd9d9d8b6aeaa060898e6d17b6
Parents: e556198
Author: Karthik Kambatla <ka...@apache.org>
Authored: Wed Mar 25 07:42:27 2015 -0700
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Wed Mar 25 07:42:27 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                  |  2 ++
 .../hadoop/yarn/client/cli/RMAdminCLI.java       |  5 +++--
 .../hadoop/yarn/client/cli/TestRMAdminCLI.java   | 19 +++++++++----------
 3 files changed, 14 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c906a1de/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 42b4662..905b414 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -79,6 +79,8 @@ Release 2.8.0 - UNRELEASED
     YARN-2868. FairScheduler: Metric for latency to allocate first container 
     for an application. (Ray Chiang via kasha)
 
+    YARN-3397. yarn rmadmin should skip -failover. (J.Andreina via kasha)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c906a1de/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
index 4642add..420eeb0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/RMAdminCLI.java
@@ -134,7 +134,8 @@ public class RMAdminCLI extends HAAdmin {
 
   private static void appendHAUsage(final StringBuilder usageBuilder) {
     for (Map.Entry<String,UsageInfo> cmdEntry : USAGE.entrySet()) {
-      if (cmdEntry.getKey().equals("-help")) {
+      if (cmdEntry.getKey().equals("-help")
+          || cmdEntry.getKey().equals("-failover")) {
         continue;
       }
       UsageInfo usageInfo = cmdEntry.getValue();
@@ -225,7 +226,7 @@ public class RMAdminCLI extends HAAdmin {
     }
     if (isHAEnabled) {
       for (String cmdKey : USAGE.keySet()) {
-        if (!cmdKey.equals("-help")) {
+        if (!cmdKey.equals("-help") && !cmdKey.equals("-failover")) {
           buildHelpMsg(cmdKey, helpBuilder);
           helpBuilder.append("\n");
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c906a1de/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
index c22494c..6067110 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestRMAdminCLI.java
@@ -364,16 +364,15 @@ public class TestRMAdminCLI {
       assertEquals(0, rmAdminCLIWithHAEnabled.run(args));
       oldOutPrintStream.println(dataOut);
       String expectedHelpMsg = 
-          "yarn rmadmin [-refreshQueues] [-refreshNodes] [-refreshSuper" +
-              "UserGroupsConfiguration] [-refreshUserToGroupsMappings] " +
-              "[-refreshAdminAcls] [-refreshServiceAcl] [-getGroup" +
-              " [username]] [[-addToClusterNodeLabels [label1,label2,label3]]" +
-              " [-removeFromClusterNodeLabels [label1,label2,label3]] [-replaceLabelsOnNode " +
-              "[node1[:port]=label1,label2 node2[:port]=label1] [-directlyAccessNodeLabelStore]] " +
-              "[-transitionToActive [--forceactive] <serviceId>] " + 
-              "[-transitionToStandby <serviceId>] [-failover" +
-              " [--forcefence] [--forceactive] <serviceId> <serviceId>] " +
-              "[-getServiceState <serviceId>] [-checkHealth <serviceId>] [-help [cmd]]";
+          "yarn rmadmin [-refreshQueues] [-refreshNodes] [-refreshSuper"
+              + "UserGroupsConfiguration] [-refreshUserToGroupsMappings] "
+              + "[-refreshAdminAcls] [-refreshServiceAcl] [-getGroup"
+              + " [username]] [[-addToClusterNodeLabels [label1,label2,label3]]"
+              + " [-removeFromClusterNodeLabels [label1,label2,label3]] [-replaceLabelsOnNode "
+              + "[node1[:port]=label1,label2 node2[:port]=label1] [-directlyAccessNodeLabelStore]] "
+              + "[-transitionToActive [--forceactive] <serviceId>] "
+              + "[-transitionToStandby <serviceId>] "
+              + "[-getServiceState <serviceId>] [-checkHealth <serviceId>] [-help [cmd]]";
       String actualHelpMsg = dataOut.toString();
       assertTrue(String.format("Help messages: %n " + actualHelpMsg + " %n doesn't include expected " +
           "messages: %n" + expectedHelpMsg), actualHelpMsg.contains(expectedHelpMsg


[37/51] [abbrv] hadoop git commit: HDFS-7875. Improve log message when wrong value configured for dfs.datanode.failed.volumes.tolerated. Contributed by Nijel.

Posted by ka...@apache.org.
HDFS-7875. Improve log message when wrong value configured for dfs.datanode.failed.volumes.tolerated. Contributed by Nijel.


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

Branch: refs/heads/YARN-2139
Commit: eda02540ce53732585b3f31411b2e65db569eb25
Parents: 51f1f49
Author: Harsh J <ha...@cloudera.com>
Authored: Tue Mar 24 23:03:30 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Tue Mar 24 23:06:18 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                    | 4 ++++
 .../hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java     | 6 ++++--
 2 files changed, 8 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/eda02540/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 4f3937a..3725a03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -321,6 +321,10 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    HDFS-7875. Improve log message when wrong value configured for
+    dfs.datanode.failed.volumes.tolerated.
+    (nijel via harsh)
+
     HDFS-2360. Ugly stacktrace when quota exceeds. (harsh)
 
     HDFS-7835. make initial sleeptime in locateFollowingBlock configurable for

http://git-wip-us.apache.org/repos/asf/hadoop/blob/eda02540/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index d42c00c..05c4871 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -276,8 +276,10 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
     this.validVolsRequired = volsConfigured - volFailuresTolerated;
 
     if (volFailuresTolerated < 0 || volFailuresTolerated >= volsConfigured) {
-      throw new DiskErrorException("Invalid volume failure "
-          + " config value: " + volFailuresTolerated);
+      throw new DiskErrorException("Invalid value configured for "
+          + "dfs.datanode.failed.volumes.tolerated - " + volFailuresTolerated
+          + ". Value configured is either less than 0 or >= "
+          + "to the number of configured volumes (" + volsConfigured + ").");
     }
     if (volsFailed > volFailuresTolerated) {
       throw new DiskErrorException("Too many failed volumes - "


[31/51] [abbrv] hadoop git commit: HDFS-3325. When configuring 'dfs.namenode.safemode.threshold-pct' to a value greater or equal to 1 there is mismatch in the UI report (Contributed by J.Andreina)

Posted by ka...@apache.org.
HDFS-3325. When configuring 'dfs.namenode.safemode.threshold-pct' to a value greater or equal to 1 there is mismatch in the UI report (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/c6c396fc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c6c396fc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c6c396fc

Branch: refs/heads/YARN-2139
Commit: c6c396fcd69514ba93583268b2633557c3d74a47
Parents: fbceb3b
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Mar 24 12:12:01 2015 +0530
Committer: Vinayakumar B <vi...@apache.org>
Committed: Tue Mar 24 12:12:01 2015 +0530

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt                      | 4 ++++
 .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java     | 2 +-
 .../src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java       | 2 +-
 .../apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java    | 2 +-
 4 files changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6c396fc/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 3ea1346..ee9a5db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -338,6 +338,10 @@ Release 2.8.0 - UNRELEASED
     HDFS-7867. Update action param from "start" to "prepare" in rolling upgrade
     javadoc (J.Andreina via vinayakumarb)
 
+    HDFS-3325. When configuring "dfs.namenode.safemode.threshold-pct" to a value
+    greater or equal to 1 there is mismatch in the UI report
+    (J.Andreina via vinayakumarb)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6c396fc/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 71c84b1..34b5e95 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
@@ -5417,7 +5417,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         msg += String.format(
           "The reported blocks %d needs additional %d"
           + " blocks to reach the threshold %.4f of total blocks %d.%n",
-          blockSafe, (blockThreshold - blockSafe) + 1, threshold, blockTotal);
+                blockSafe, (blockThreshold - blockSafe), threshold, blockTotal);
         thresholdsMet = false;
       } else {
         msg += String.format("The reported blocks %d has reached the threshold"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6c396fc/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 2d5bef2..80fe9ee 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
@@ -198,7 +198,7 @@ public class TestSafeMode {
     
     String status = nn.getNamesystem().getSafemode();
     assertEquals("Safe mode is ON. The reported blocks 0 needs additional " +
-        "15 blocks to reach the threshold 0.9990 of total blocks 15." + NEWLINE +
+        "14 blocks to reach the threshold 0.9990 of total blocks 15." + NEWLINE +
         "The number of live datanodes 0 has reached the minimum number 0. " +
         "Safe mode will be turned off automatically once the thresholds " +
         "have been reached.", status);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c6c396fc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java
index c5aad9c..86f3e7b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHASafeMode.java
@@ -498,7 +498,7 @@ public class TestHASafeMode {
             + nodeThresh + ". In safe mode extension. "
             + "Safe mode will be turned off automatically"));
     } else {
-      int additional = total - safe;
+      int additional = (int) (total * 0.9990) - safe;
       assertTrue("Bad safemode status: '" + status + "'",
           status.startsWith(
               "Safe mode is ON. " +


[45/51] [abbrv] hadoop git commit: HADOOP-11741. Add LOG.isDebugEnabled() guard for some LOG.debug(). Contributed by Walter Su.

Posted by ka...@apache.org.
HADOOP-11741. Add LOG.isDebugEnabled() guard for some LOG.debug(). 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/5582b0f1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5582b0f1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5582b0f1

Branch: refs/heads/YARN-2139
Commit: 5582b0f1d469e7c98811a341c4b4c78eaa64ede5
Parents: 80278a5
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed Mar 25 16:36:10 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Wed Mar 25 16:36:10 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 +
 .../org/apache/hadoop/conf/Configuration.java   |  4 +-
 .../hadoop/crypto/key/JavaKeyStoreProvider.java | 14 +++--
 .../apache/hadoop/ha/ActiveStandbyElector.java  | 65 +++++++++++++-------
 .../main/java/org/apache/hadoop/ipc/RPC.java    |  9 ++-
 .../main/java/org/apache/hadoop/ipc/Server.java |  8 ++-
 .../metrics/ganglia/GangliaContext31.java       |  6 +-
 .../security/ssl/FileBasedKeyStoresFactory.java | 22 +++++--
 8 files changed, 90 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5582b0f1/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 cdb88d2..e3cadf5 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -459,6 +459,9 @@ Release 2.8.0 - UNRELEASED
     HADOOP-11737. mockito's version in hadoop-nfs’ pom.xml shouldn't be
     specified. (Kengo Seki via ozawa)
 
+    HADOOP-11741. Add LOG.isDebugEnabled() guard for some LOG.debug().
+    (Walter Su via ozawa)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5582b0f1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
index 753f515..8a312ff 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -2479,7 +2479,9 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
   private Document parse(DocumentBuilder builder, URL url)
       throws IOException, SAXException {
     if (!quietmode) {
-      LOG.debug("parsing URL " + url);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("parsing URL " + url);
+      }
     }
     if (url == null) {
       return null;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5582b0f1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
index c0d510d..091cab5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
@@ -214,9 +214,11 @@ public class JavaKeyStoreProvider extends KeyProvider {
         renameOrFail(path, new Path(path.toString() + "_CORRUPTED_"
             + System.currentTimeMillis()));
         renameOrFail(backupPath, path);
-        LOG.debug(String.format(
-            "KeyStore loaded successfully from '%s' since '%s'"
-                + "was corrupted !!", backupPath, path));
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(String.format(
+              "KeyStore loaded successfully from '%s' since '%s'"
+                  + "was corrupted !!", backupPath, path));
+        }
       } else {
         throw ioe;
       }
@@ -265,8 +267,10 @@ public class JavaKeyStoreProvider extends KeyProvider {
     try {
       perm = loadFromPath(pathToLoad, password);
       renameOrFail(pathToLoad, path);
-      LOG.debug(String.format("KeyStore loaded successfully from '%s'!!",
-          pathToLoad));
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(String.format("KeyStore loaded successfully from '%s'!!",
+            pathToLoad));
+      }
       if (fs.exists(pathToDelete)) {
         fs.delete(pathToDelete, true);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5582b0f1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
index 947baa9..e520a16 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java
@@ -256,7 +256,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
     appData = new byte[data.length];
     System.arraycopy(data, 0, appData, 0, data.length);
 
-    LOG.debug("Attempting active election for " + this);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Attempting active election for " + this);
+    }
     joinElectionInternal();
   }
   
@@ -406,9 +408,11 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
   public synchronized void processResult(int rc, String path, Object ctx,
       String name) {
     if (isStaleClient(ctx)) return;
-    LOG.debug("CreateNode result: " + rc + " for path: " + path
-        + " connectionState: " + zkConnectionState +
-        "  for " + this);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("CreateNode result: " + rc + " for path: " + path
+          + " connectionState: " + zkConnectionState +
+          "  for " + this);
+    }
 
     Code code = Code.get(rc);
     if (isSuccess(code)) {
@@ -467,10 +471,11 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
     
     assert wantToBeInElection :
         "Got a StatNode result after quitting election";
-    
-    LOG.debug("StatNode result: " + rc + " for path: " + path
-        + " connectionState: " + zkConnectionState + " for " + this);
-        
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("StatNode result: " + rc + " for path: " + path
+          + " connectionState: " + zkConnectionState + " for " + this);
+    }
 
     Code code = Code.get(rc);
     if (isSuccess(code)) {
@@ -535,10 +540,12 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
   synchronized void processWatchEvent(ZooKeeper zk, WatchedEvent event) {
     Event.EventType eventType = event.getType();
     if (isStaleClient(zk)) return;
-    LOG.debug("Watcher event type: " + eventType + " with state:"
-        + event.getState() + " for path:" + event.getPath()
-        + " connectionState: " + zkConnectionState
-        + " for " + this);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Watcher event type: " + eventType + " with state:"
+          + event.getState() + " for path:" + event.getPath()
+          + " connectionState: " + zkConnectionState
+          + " for " + this);
+    }
 
     if (eventType == Event.EventType.None) {
       // the connection state has changed
@@ -597,7 +604,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
         monitorActiveStatus();
         break;
       default:
-        LOG.debug("Unexpected node event: " + eventType + " for path: " + path);
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Unexpected node event: " + eventType + " for path: " + path);
+        }
         monitorActiveStatus();
       }
 
@@ -646,7 +655,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
 
   private void monitorActiveStatus() {
     assert wantToBeInElection;
-    LOG.debug("Monitoring active leader for " + this);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Monitoring active leader for " + this);
+    }
     statRetryCount = 0;
     monitorLockNodeAsync();
   }
@@ -737,7 +748,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
     int connectionRetryCount = 0;
     boolean success = false;
     while(!success && connectionRetryCount < maxRetryNum) {
-      LOG.debug("Establishing zookeeper connection for " + this);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Establishing zookeeper connection for " + this);
+      }
       try {
         createConnection();
         success = true;
@@ -765,7 +778,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
       watcher = null;
     }
     zkClient = getNewZooKeeper();
-    LOG.debug("Created new connection for " + this);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Created new connection for " + this);
+    }
   }
 
   @InterfaceAudience.Private
@@ -773,7 +788,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
     if (zkClient == null) {
       return;
     }
-    LOG.debug("Terminating ZK connection for " + this);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Terminating ZK connection for " + this);
+    }
     ZooKeeper tempZk = zkClient;
     zkClient = null;
     watcher = null;
@@ -800,8 +817,10 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
     try {
       Stat oldBreadcrumbStat = fenceOldActive();
       writeBreadCrumbNode(oldBreadcrumbStat);
-      
-      LOG.debug("Becoming active for " + this);
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Becoming active for " + this);
+      }
       appClient.becomeActive();
       state = State.ACTIVE;
       return true;
@@ -906,7 +925,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
 
   private void becomeStandby() {
     if (state != State.STANDBY) {
-      LOG.debug("Becoming standby for " + this);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Becoming standby for " + this);
+      }
       state = State.STANDBY;
       appClient.becomeStandby();
     }
@@ -914,7 +935,9 @@ public class ActiveStandbyElector implements StatCallback, StringCallback {
 
   private void enterNeutralMode() {
     if (state != State.NEUTRAL) {
-      LOG.debug("Entering neutral mode for " + this);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Entering neutral mode for " + this);
+      }
       state = State.NEUTRAL;
       appClient.enterNeutralMode();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5582b0f1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
index 8ada0ff..4766175 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java
@@ -876,9 +876,12 @@ public class RPC {
 
      getProtocolImplMap(rpcKind).put(new ProtoNameVer(protocolName, version),
          new ProtoClassProtoImpl(protocolClass, protocolImpl)); 
-     LOG.debug("RpcKind = " + rpcKind + " Protocol Name = " + protocolName +  " version=" + version +
-         " ProtocolImpl=" + protocolImpl.getClass().getName() + 
-         " protocolClass=" + protocolClass.getName());
+     if (LOG.isDebugEnabled()) {
+       LOG.debug("RpcKind = " + rpcKind + " Protocol Name = " + protocolName +
+           " version=" + version +
+           " ProtocolImpl=" + protocolImpl.getClass().getName() +
+           " protocolClass=" + protocolClass.getName());
+     }
    }
    
    static class VerProtocolImpl {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5582b0f1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
index d2d61b3..9aa362e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java
@@ -234,9 +234,11 @@ public abstract class Server {
       throw new IllegalArgumentException("ReRegistration of rpcKind: " +
           rpcKind);      
     }
-    LOG.debug("rpcKind=" + rpcKind + 
-        ", rpcRequestWrapperClass=" + rpcRequestWrapperClass + 
-        ", rpcInvoker=" + rpcInvoker);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("rpcKind=" + rpcKind +
+          ", rpcRequestWrapperClass=" + rpcRequestWrapperClass +
+          ", rpcInvoker=" + rpcInvoker);
+    }
   }
   
   public Class<? extends Writable> getRpcRequestWrapper(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5582b0f1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext31.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext31.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext31.java
index f35ad18..9ad2d39 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext31.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics/ganglia/GangliaContext31.java
@@ -82,8 +82,10 @@ public class GangliaContext31 extends GangliaContext {
       return;
     }
 
-    LOG.debug("Emitting metric " + name + ", type " + type + ", value " + 
-      value + " from hostname" + hostName);
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Emitting metric " + name + ", type " + type + ", value " +
+          value + " from hostname" + hostName);
+    }
 
     String units = getUnits(name);
     int slope = getSlope(name);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5582b0f1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java
index 609c71f..41634a8 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/FileBasedKeyStoresFactory.java
@@ -164,7 +164,9 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
       // configuration property for key password.
       keystoreKeyPassword = getPassword(
           conf, keyPasswordProperty, keystorePassword);
-      LOG.debug(mode.toString() + " KeyStore: " + keystoreLocation);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(mode.toString() + " KeyStore: " + keystoreLocation);
+      }
 
       InputStream is = new FileInputStream(keystoreLocation);
       try {
@@ -172,7 +174,9 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
       } finally {
         is.close();
       }
-      LOG.debug(mode.toString() + " Loaded KeyStore: " + keystoreLocation);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(mode.toString() + " Loaded KeyStore: " + keystoreLocation);
+      }
     } else {
       keystore.load(null, null);
     }
@@ -204,18 +208,24 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
               resolvePropertyName(mode, SSL_TRUSTSTORE_RELOAD_INTERVAL_TPL_KEY),
               DEFAULT_SSL_TRUSTSTORE_RELOAD_INTERVAL);
 
-      LOG.debug(mode.toString() + " TrustStore: " + truststoreLocation);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(mode.toString() + " TrustStore: " + truststoreLocation);
+      }
 
       trustManager = new ReloadingX509TrustManager(truststoreType,
           truststoreLocation,
           truststorePassword,
           truststoreReloadInterval);
       trustManager.init();
-      LOG.debug(mode.toString() + " Loaded TrustStore: " + truststoreLocation);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(mode.toString() + " Loaded TrustStore: " + truststoreLocation);
+      }
       trustManagers = new TrustManager[]{trustManager};
     } else {
-      LOG.debug("The property '" + locationProperty + "' has not been set, " +
-          "no TrustStore will be loaded");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("The property '" + locationProperty + "' has not been set, " +
+            "no TrustStore will be loaded");
+      }
       trustManagers = null;
     }
   }


[46/51] [abbrv] hadoop git commit: HADOOP-11014. Potential resource leak in JavaKeyStoreProvider due to unclosed stream. (ozawa)

Posted by ka...@apache.org.
HADOOP-11014. Potential resource leak in JavaKeyStoreProvider due to unclosed stream. (ozawa)


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

Branch: refs/heads/YARN-2139
Commit: b351086ff66ca279c0550e078e3a9d110f3f36a5
Parents: 5582b0f
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Wed Mar 25 16:59:40 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Wed Mar 25 16:59:40 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt      |  3 +++
 .../hadoop/crypto/key/JavaKeyStoreProvider.java      | 15 ++++++++-------
 .../hadoop/security/alias/JavaKeyStoreProvider.java  | 15 +++++++--------
 3 files changed, 18 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b351086f/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 e3cadf5..0c311df 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -1142,6 +1142,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11609. Correct credential commands info in
     CommandsManual.html#credential. (Varun Saxena via ozawa)
 
+    HADOOP-11014. Potential resource leak in JavaKeyStoreProvider due to
+    unclosed stream. (ozawa)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b351086f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
index 091cab5..c6d60a3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/JavaKeyStoreProvider.java
@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -303,9 +304,11 @@ public class JavaKeyStoreProvider extends KeyProvider {
 
   private FsPermission loadFromPath(Path p, char[] password)
       throws IOException, NoSuchAlgorithmException, CertificateException {
-    FileStatus s = fs.getFileStatus(p);
-    keyStore.load(fs.open(p), password);
-    return s.getPermission();
+    try (FSDataInputStream in = fs.open(p)) {
+      FileStatus s = fs.getFileStatus(p);
+      keyStore.load(in, password);
+      return s.getPermission();
+    }
   }
 
   private Path constructNewPath(Path path) {
@@ -599,9 +602,8 @@ public class JavaKeyStoreProvider extends KeyProvider {
   }
 
   protected void writeToNew(Path newPath) throws IOException {
-    FSDataOutputStream out =
-        FileSystem.create(fs, newPath, permissions);
-    try {
+    try (FSDataOutputStream out =
+        FileSystem.create(fs, newPath, permissions);) {
       keyStore.store(out, password);
     } catch (KeyStoreException e) {
       throw new IOException("Can't store keystore " + this, e);
@@ -612,7 +614,6 @@ public class JavaKeyStoreProvider extends KeyProvider {
       throw new IOException(
           "Certificate exception storing keystore " + this, e);
     }
-    out.close();
   }
 
   protected boolean backupToOld(Path oldPath)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b351086f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java
index 05958a0..5e5cebb 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/JavaKeyStoreProvider.java
@@ -22,6 +22,7 @@ import org.apache.commons.io.Charsets;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -98,11 +99,8 @@ public class JavaKeyStoreProvider extends CredentialProvider {
         ClassLoader cl = Thread.currentThread().getContextClassLoader();
         URL pwdFile = cl.getResource(pwFile);
         if (pwdFile != null) {
-          InputStream is = pwdFile.openStream();
-          try {
+          try (InputStream is = pwdFile.openStream()) {
             password = IOUtils.toString(is).trim().toCharArray();
-          } finally {
-            is.close();
           }
         }
       }
@@ -110,6 +108,7 @@ public class JavaKeyStoreProvider extends CredentialProvider {
     if (password == null) {
       password = KEYSTORE_PASSWORD_DEFAULT.toCharArray();
     }
+
     try {
       keyStore = KeyStore.getInstance(SCHEME_NAME);
       if (fs.exists(path)) {
@@ -118,7 +117,9 @@ public class JavaKeyStoreProvider extends CredentialProvider {
         FileStatus s = fs.getFileStatus(path);
         permissions = s.getPermission();
 
-        keyStore.load(fs.open(path), password);
+        try (FSDataInputStream in = fs.open(path)) {
+          keyStore.load(in, password);
+        }
       } else {
         permissions = new FsPermission("700");
         // required to create an empty keystore. *sigh*
@@ -257,8 +258,7 @@ public class JavaKeyStoreProvider extends CredentialProvider {
         return;
       }
       // write out the keystore
-      FSDataOutputStream out = FileSystem.create(fs, path, permissions);
-      try {
+      try (FSDataOutputStream out = FileSystem.create(fs, path, permissions)) {
         keyStore.store(out, password);
       } catch (KeyStoreException e) {
         throw new IOException("Can't store keystore " + this, e);
@@ -268,7 +268,6 @@ public class JavaKeyStoreProvider extends CredentialProvider {
         throw new IOException("Certificate exception storing keystore " + this,
             e);
       }
-      out.close();
       changed = false;
     }
     finally {


[09/51] [abbrv] hadoop git commit: YARN-3345. Add non-exclusive node label API. Contributed by Wangda Tan

Posted by ka...@apache.org.
YARN-3345. Add non-exclusive node label API. Contributed by Wangda Tan


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

Branch: refs/heads/YARN-2139
Commit: e1feb4ea1a532d680d6ca69b55ffcae1552d64f0
Parents: 7f1e2f9
Author: Jian He <ji...@apache.org>
Authored: Fri Mar 20 19:04:38 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Fri Mar 20 19:04:38 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   2 +
 .../hadoop/yarn/api/records/NodeLabel.java      |  55 ++++
 .../ResourceManagerAdministrationProtocol.java  |  12 +-
 .../UpdateNodeLabelsRequest.java                |  49 +++
 .../UpdateNodeLabelsResponse.java               |  37 +++
 ...esourcemanager_administration_protocol.proto |   1 +
 ..._server_resourcemanager_service_protos.proto |   8 +
 .../src/main/proto/yarn_protos.proto            |   5 +
 .../api/records/impl/pb/NodeLabelPBImpl.java    | 106 +++++++
 .../nodelabels/CommonNodeLabelsManager.java     |  75 ++++-
 .../nodelabels/FileSystemNodeLabelsStore.java   |  28 +-
 .../hadoop/yarn/nodelabels/NodeLabel.java       | 113 -------
 .../hadoop/yarn/nodelabels/NodeLabelsStore.java |  11 +-
 .../hadoop/yarn/nodelabels/RMNodeLabel.java     | 122 ++++++++
 .../event/NodeLabelsStoreEventType.java         |   3 +-
 .../event/StoreUpdateNodeLabelsEvent.java       |  36 +++
 ...nagerAdministrationProtocolPBClientImpl.java |  19 ++
 ...agerAdministrationProtocolPBServiceImpl.java |  23 ++
 .../impl/pb/UpdateNodeLabelsRequestPBImpl.java  | 145 +++++++++
 .../impl/pb/UpdateNodeLabelsResponsePBImpl.java |  67 ++++
 .../hadoop/yarn/api/TestPBImplRecords.java      | 302 ++++++++++++++++++-
 .../DummyCommonNodeLabelsManager.java           |   9 +
 .../nodelabels/TestCommonNodeLabelsManager.java |  28 ++
 .../TestFileSystemNodeLabelsStore.java          |  15 +-
 .../server/resourcemanager/AdminService.java    |  30 +-
 .../nodelabels/RMNodeLabelsManager.java         |  24 +-
 .../resourcemanager/webapp/NodeLabelsPage.java  |   4 +-
 .../nodelabels/NullRMNodeLabelsManager.java     |   8 +
 .../nodelabels/TestRMNodeLabelsManager.java     |   8 +-
 29 files changed, 1189 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 177d587..f8c1a76 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -54,6 +54,8 @@ Release 2.8.0 - UNRELEASED
 
   NEW FEATURES
 
+    YARN-3345. Add non-exclusive node label API. (Wangda Tan via jianhe)
+
   IMPROVEMENTS
 
     YARN-3243. CapacityScheduler should pass headroom from parent to children

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeLabel.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeLabel.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeLabel.java
new file mode 100644
index 0000000..23da1f4
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/NodeLabel.java
@@ -0,0 +1,55 @@
+/**
+ * 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.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+@Public
+@Unstable
+public abstract class NodeLabel {
+  @Public
+  @Unstable
+  public static NodeLabel newInstance(String nodeLabel,
+      boolean isExclusive) {
+    NodeLabel request =
+        Records.newRecord(NodeLabel.class);
+    request.setNodeLabel(nodeLabel);
+    request.setIsExclusive(isExclusive);
+    return request;
+  }
+  
+  @Public
+  @Stable
+  public abstract String getNodeLabel();
+  
+  @Public
+  @Unstable
+  public abstract void setNodeLabel(String nodeLabel);
+  
+  @Public
+  @Stable
+  public abstract boolean getIsExclusive();
+  
+  @Public
+  @Unstable
+  public abstract void setIsExclusive(boolean isExclusive);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
index 2061aef..8777e00 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ResourceManagerAdministrationProtocol.java
@@ -48,6 +48,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLa
 import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
 
@@ -120,8 +122,8 @@ public interface ResourceManagerAdministrationProtocol extends GetUserMappingsPr
   @Public
   @Evolving
   @Idempotent
-  public AddToClusterNodeLabelsResponse addToClusterNodeLabels(AddToClusterNodeLabelsRequest request)
-      throws YarnException, IOException;
+  public AddToClusterNodeLabelsResponse addToClusterNodeLabels(
+      AddToClusterNodeLabelsRequest request) throws YarnException, IOException;
    
   @Public
   @Evolving
@@ -134,4 +136,10 @@ public interface ResourceManagerAdministrationProtocol extends GetUserMappingsPr
   @Idempotent
   public ReplaceLabelsOnNodeResponse replaceLabelsOnNode(
       ReplaceLabelsOnNodeRequest request) throws YarnException, IOException;
+  
+  @Public
+  @Evolving
+  @Idempotent
+  public UpdateNodeLabelsResponse updateNodeLabels(
+      UpdateNodeLabelsRequest request) throws YarnException, IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeLabelsRequest.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeLabelsRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeLabelsRequest.java
new file mode 100644
index 0000000..44bdc65
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeLabelsRequest.java
@@ -0,0 +1,49 @@
+/**
+ * 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.yarn.server.api.protocolrecords;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.util.Records;
+
+@Public
+@Unstable
+public abstract class UpdateNodeLabelsRequest {
+  @Public
+  @Unstable
+  public static UpdateNodeLabelsRequest newInstance(
+      List<NodeLabel> NodeLabels) {
+    UpdateNodeLabelsRequest request =
+      Records.newRecord(UpdateNodeLabelsRequest.class);
+    request.setNodeLabels(NodeLabels);
+    return request;
+  }
+
+  @Public
+  @Unstable
+  public abstract void setNodeLabels(
+      List<NodeLabel> NodeLabels);
+  
+  @Public
+  @Unstable
+  public abstract List<NodeLabel> getNodeLabels();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeLabelsResponse.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeLabelsResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeLabelsResponse.java
new file mode 100644
index 0000000..eb704c0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/UpdateNodeLabelsResponse.java
@@ -0,0 +1,37 @@
+/**
+ * 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.yarn.server.api.protocolrecords;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+@Public
+@Unstable
+public class UpdateNodeLabelsResponse {
+  
+  @Private
+  @Unstable
+  public static UpdateNodeLabelsResponse newInstance() {
+    UpdateNodeLabelsResponse response =
+        Records.newRecord(UpdateNodeLabelsResponse.class);
+    return response;  
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
index 7f54b8e..6646718 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/resourcemanager_administration_protocol.proto
@@ -42,4 +42,5 @@ service ResourceManagerAdministrationProtocolService {
   rpc addToClusterNodeLabels(AddToClusterNodeLabelsRequestProto) returns (AddToClusterNodeLabelsResponseProto);
   rpc removeFromClusterNodeLabels(RemoveFromClusterNodeLabelsRequestProto) returns (RemoveFromClusterNodeLabelsResponseProto);
   rpc replaceLabelsOnNodes(ReplaceLabelsOnNodeRequestProto) returns (ReplaceLabelsOnNodeResponseProto);
+  rpc updateNodeLabels(UpdateNodeLabelsRequestProto) returns (UpdateNodeLabelsResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
index 900e349..0d5b5c7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/server/yarn_server_resourcemanager_service_protos.proto
@@ -97,6 +97,14 @@ message ReplaceLabelsOnNodeResponseProto {
   
 }
 
+message UpdateNodeLabelsRequestProto {
+  repeated NodeLabelProto nodeLabels = 1;
+}
+
+
+message UpdateNodeLabelsResponseProto {
+}
+
 //////////////////////////////////////////////////////////////////
 ///////////// RM Failover related records ////////////////////////
 //////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
index 2edff99..194be82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
@@ -244,6 +244,11 @@ message LabelsToNodeIdsProto {
   repeated NodeIdProto nodeId = 2;
 }
 
+message NodeLabelProto {
+  optional string nodeLabel = 1;
+  optional bool isExclusive = 2 [default = true]; 
+}
+
 ////////////////////////////////////////////////////////////////////////
 ////// From AM_RM_Protocol /////////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeLabelPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeLabelPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeLabelPBImpl.java
new file mode 100644
index 0000000..661359e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/NodeLabelPBImpl.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.yarn.api.records.impl.pb;
+
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProtoOrBuilder;
+
+public class NodeLabelPBImpl extends NodeLabel {
+  NodeLabelProto proto = 
+      NodeLabelProto.getDefaultInstance();
+  NodeLabelProto.Builder builder = null;
+  boolean viaProto = false;
+  
+  public NodeLabelPBImpl() {
+    builder = NodeLabelProto.newBuilder();
+  }
+
+  public NodeLabelPBImpl(NodeLabelProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+  
+  public NodeLabelProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+  
+  private void mergeLocalToProto() {
+    if (viaProto) 
+      maybeInitBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+  
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+  
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = NodeLabelProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public String getNodeLabel() {
+    NodeLabelProtoOrBuilder p = viaProto ? proto : builder;
+    if (!p.hasNodeLabel()) {
+      return null;
+    }
+    return (p.getNodeLabel());
+  }
+
+  @Override
+  public void setNodeLabel(String nodeLabel) {
+    maybeInitBuilder();
+    if (nodeLabel == null) {
+      builder.clearNodeLabel();
+      return;
+    }
+    builder.setNodeLabel(nodeLabel);
+  }
+
+  @Override
+  public boolean getIsExclusive() {
+    NodeLabelProtoOrBuilder p = viaProto ? proto : builder;
+    return p.getIsExclusive();
+  }
+
+  @Override
+  public void setIsExclusive(boolean isExclusive) {
+    maybeInitBuilder();
+    builder.setIsExclusive(isExclusive);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
index 67a5494..a5e2756 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/CommonNodeLabelsManager.java
@@ -24,6 +24,7 @@ import java.util.Collections;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -40,16 +41,19 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEvent;
 import org.apache.hadoop.yarn.nodelabels.event.NodeLabelsStoreEventType;
 import org.apache.hadoop.yarn.nodelabels.event.RemoveClusterNodeLabels;
 import org.apache.hadoop.yarn.nodelabels.event.StoreNewClusterNodeLabels;
+import org.apache.hadoop.yarn.nodelabels.event.StoreUpdateNodeLabelsEvent;
 import org.apache.hadoop.yarn.nodelabels.event.UpdateNodeToLabelsMappingsEvent;
 import org.apache.hadoop.yarn.util.resource.Resources;
 
@@ -83,8 +87,8 @@ public class CommonNodeLabelsManager extends AbstractService {
 
   protected Dispatcher dispatcher;
 
-  protected ConcurrentMap<String, NodeLabel> labelCollections =
-      new ConcurrentHashMap<String, NodeLabel>();
+  protected ConcurrentMap<String, RMNodeLabel> labelCollections =
+      new ConcurrentHashMap<String, RMNodeLabel>();
   protected ConcurrentMap<String, Host> nodeCollections =
       new ConcurrentHashMap<String, Host>();
 
@@ -181,6 +185,13 @@ public class CommonNodeLabelsManager extends AbstractService {
         store.updateNodeToLabelsMappings(updateNodeToLabelsMappingsEvent
             .getNodeToLabels());
         break;
+      case UPDATE_NODE_LABELS:
+        StoreUpdateNodeLabelsEvent
+          storeSetNodeLabelsEventEvent =
+            (StoreUpdateNodeLabelsEvent) event;
+        store.updateNodeLabels(storeSetNodeLabelsEventEvent
+            .getUpdatedNodeLabels());
+        break;
       }
     } catch (IOException e) {
       LOG.error("Failed to store label modification to storage");
@@ -214,7 +225,7 @@ public class CommonNodeLabelsManager extends AbstractService {
       initNodeLabelStore(conf);
     }
     
-    labelCollections.put(NO_LABEL, new NodeLabel(NO_LABEL));
+    labelCollections.put(NO_LABEL, new RMNodeLabel(NO_LABEL));
   }
 
   protected void initNodeLabelStore(Configuration conf) throws Exception {
@@ -288,7 +299,7 @@ public class CommonNodeLabelsManager extends AbstractService {
     for (String label : labels) {
       // shouldn't overwrite it to avoid changing the Label.resource
       if (this.labelCollections.get(label) == null) {
-        this.labelCollections.put(label, new NodeLabel(label));
+        this.labelCollections.put(label, new RMNodeLabel(label));
         newLabels.add(label);
       }
     }
@@ -746,7 +757,7 @@ public class CommonNodeLabelsManager extends AbstractService {
         if(label.equals(NO_LABEL)) {
           continue;
         }
-        NodeLabel nodeLabelInfo = labelCollections.get(label);
+        RMNodeLabel nodeLabelInfo = labelCollections.get(label);
         if(nodeLabelInfo != null) {
           Set<NodeId> nodeIds = nodeLabelInfo.getAssociatedNodeIds();
           if (!nodeIds.isEmpty()) {
@@ -777,6 +788,60 @@ public class CommonNodeLabelsManager extends AbstractService {
       readLock.unlock();
     }
   }
+  
+  private void checkUpdateNodeLabels(
+      List<NodeLabel> updatedNodeLabels) throws YarnException {
+    // pre-check
+    for (NodeLabel label : updatedNodeLabels) {
+      if (!labelCollections.containsKey(label.getNodeLabel())) {
+        String message =
+          String.format(
+            "Trying to update a non-existing node-label=%s",
+            label.getNodeLabel());
+        LOG.error(message);
+        throw new YarnException(message);
+      }
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  public void updateNodeLabels(
+      List<NodeLabel> updatedNodeLabels) throws YarnException {
+    try {
+      writeLock.lock();
+      checkUpdateNodeLabels(updatedNodeLabels);
+
+      for (NodeLabel label : updatedNodeLabels) {
+        RMNodeLabel rmLabel = labelCollections.get(label.getNodeLabel());
+        rmLabel.setIsExclusive(label.getIsExclusive());
+      }
+
+      if (null != dispatcher && !updatedNodeLabels.isEmpty()) {
+        dispatcher.getEventHandler().handle(
+          new StoreUpdateNodeLabelsEvent(updatedNodeLabels));
+      }
+    } finally {
+      writeLock.unlock();
+    }
+  }
+
+  public boolean isExclusiveNodeLabel(String nodeLabel) throws IOException {
+    try {
+      readLock.lock();
+      RMNodeLabel label = labelCollections.get(nodeLabel);
+      if (label == null) {
+        String message =
+          "Getting is-exclusive-node-label, node-label = " + nodeLabel
+            + ", is not existed.";
+        LOG.error(message);
+        throw new IOException(message);
+      }
+      return label.getIsExclusive();
+    } finally {
+      readLock.unlock();
+    }
+  }
+  
 
   private void checkAndThrowLabelName(String label) throws IOException {
     if (label == null || label.isEmpty() || label.length() > MAX_LABEL_LENGTH) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
index 6e685ee..0c7192f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/FileSystemNodeLabelsStore.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.nodelabels;
 import java.io.EOFException;
 import java.io.IOException;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
@@ -34,16 +35,21 @@ import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RemoveFromClusterNodeLabelsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeLabelsRequestProto;
 import org.apache.hadoop.yarn.server.api.protocolrecords.AddToClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeLabelsRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RemoveFromClusterNodeLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeLabelsRequestPBImpl;
 
 import com.google.common.collect.Sets;
 
@@ -60,7 +66,7 @@ public class FileSystemNodeLabelsStore extends NodeLabelsStore {
   protected static final String EDITLOG_FILENAME = "nodelabel.editlog";
   
   protected enum SerializedLogType {
-    ADD_LABELS, NODE_TO_LABELS, REMOVE_LABELS
+    ADD_LABELS, NODE_TO_LABELS, REMOVE_LABELS, UPDATE_NODE_LABELS
   }
 
   Path fsWorkingPath;
@@ -150,9 +156,19 @@ public class FileSystemNodeLabelsStore extends NodeLabelsStore {
         .newHashSet(labels.iterator()))).getProto().writeDelimitedTo(editlogOs);
     ensureCloseEditlogFile();
   }
+  
+  @Override
+  public void updateNodeLabels(List<NodeLabel> updatedNodeLabels)
+      throws IOException {
+    ensureAppendEditlogFile();
+    editlogOs.writeInt(SerializedLogType.UPDATE_NODE_LABELS.ordinal());
+    ((UpdateNodeLabelsRequestPBImpl) UpdateNodeLabelsRequest
+        .newInstance(updatedNodeLabels)).getProto().writeDelimitedTo(editlogOs);
+    ensureCloseEditlogFile();
+  }
 
   @Override
-  public void recover() throws IOException {
+  public void recover() throws YarnException, IOException {
     /*
      * Steps of recover
      * 1) Read from last mirror (from mirror or mirror.old)
@@ -221,6 +237,14 @@ public class FileSystemNodeLabelsStore extends NodeLabelsStore {
             mgr.replaceLabelsOnNode(map);
             break;
           }
+          case UPDATE_NODE_LABELS: {
+            List<NodeLabel> attributes =
+                new UpdateNodeLabelsRequestPBImpl(
+                    UpdateNodeLabelsRequestProto.parseDelimitedFrom(is))
+                    .getNodeLabels();
+            mgr.updateNodeLabels(attributes);
+            break;
+          }
           }
         } catch (EOFException e) {
           // EOF hit, break

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabel.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabel.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabel.java
deleted file mode 100644
index 1765a65..0000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabel.java
+++ /dev/null
@@ -1,113 +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.yarn.nodelabels;
-
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.commons.lang.StringUtils;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.util.resource.Resources;
-
-public class NodeLabel implements Comparable<NodeLabel> {
-  private Resource resource;
-  private int numActiveNMs;
-  private String labelName;
-  private Set<NodeId> nodeIds;
-
-  public NodeLabel(String labelName) {
-    this(labelName, Resource.newInstance(0, 0), 0);
-  }
-  
-  protected NodeLabel(String labelName, Resource res, int activeNMs) {
-    this.labelName = labelName;
-    this.resource = res;
-    this.numActiveNMs = activeNMs;
-    this.nodeIds = new HashSet<NodeId>();
-  }
-
-  public void addNodeId(NodeId node) {
-    nodeIds.add(node);
-  }
-
-  public void removeNodeId(NodeId node) {
-    nodeIds.remove(node);
-  }
-  
-  public Set<NodeId> getAssociatedNodeIds() {
-    return new HashSet<NodeId>(nodeIds);
-  }
-
-  public void addNode(Resource nodeRes) {
-    Resources.addTo(resource, nodeRes);
-    numActiveNMs++;
-  }
-  
-  public void removeNode(Resource nodeRes) {
-    Resources.subtractFrom(resource, nodeRes);
-    numActiveNMs--;
-  }
-
-  public Resource getResource() {
-    return this.resource;
-  }
-
-  public int getNumActiveNMs() {
-    return numActiveNMs;
-  }
-  
-  public String getLabelName() {
-    return labelName;
-  }
-  
-  public NodeLabel getCopy() {
-    return new NodeLabel(labelName, resource, numActiveNMs);
-  }
-  
-  @Override
-  public int compareTo(NodeLabel o) {
-    // We should always put empty label entry first after sorting
-    if (labelName.isEmpty() != o.getLabelName().isEmpty()) {
-      if (labelName.isEmpty()) {
-        return -1;
-      }
-      return 1;
-    }
-    
-    return labelName.compareTo(o.getLabelName());
-  }
-  
-  @Override
-  public boolean equals(Object obj) {
-    if (obj instanceof NodeLabel) {
-      NodeLabel other = (NodeLabel) obj;
-      return Resources.equals(resource, other.getResource())
-          && StringUtils.equals(labelName, other.getLabelName())
-          && (other.getNumActiveNMs() == numActiveNMs); 
-    }
-    return false;
-  }
-  
-  @Override
-  public int hashCode() {
-    final int prime = 502357;
-    return (int) ((((long) labelName.hashCode() << 8)
-        + (resource.hashCode() << 4) + numActiveNMs) % prime);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
index 4a34a09..6bd90db 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/NodeLabelsStore.java
@@ -21,11 +21,14 @@ package org.apache.hadoop.yarn.nodelabels;
 import java.io.Closeable;
 import java.io.IOException;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 
 public abstract class NodeLabelsStore implements Closeable {
   protected final CommonNodeLabelsManager mgr;
@@ -53,9 +56,15 @@ public abstract class NodeLabelsStore implements Closeable {
       throws IOException;
   
   /**
+   * Update node labels
+   */
+  public abstract void updateNodeLabels(
+      List<NodeLabel> updatedNodeLabels) throws IOException;
+  
+  /**
    * Recover labels and node to labels mappings from store
    */
-  public abstract void recover() throws IOException;
+  public abstract void recover() throws IOException, YarnException;
   
   public void init(Configuration conf) throws Exception {}
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
new file mode 100644
index 0000000..7638e21
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/RMNodeLabel.java
@@ -0,0 +1,122 @@
+/**
+ * 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.yarn.nodelabels;
+
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.resource.Resources;
+
+public class RMNodeLabel implements Comparable<RMNodeLabel> {
+  private Resource resource;
+  private int numActiveNMs;
+  private String labelName;
+  private Set<NodeId> nodeIds;
+  private boolean exclusive = true;
+
+  public RMNodeLabel(String labelName) {
+    this(labelName, Resource.newInstance(0, 0), 0);
+  }
+  
+  protected RMNodeLabel(String labelName, Resource res, int activeNMs) {
+    this.labelName = labelName;
+    this.resource = res;
+    this.numActiveNMs = activeNMs;
+    this.nodeIds = new HashSet<NodeId>();
+  }
+
+  public void addNodeId(NodeId node) {
+    nodeIds.add(node);
+  }
+
+  public void removeNodeId(NodeId node) {
+    nodeIds.remove(node);
+  }
+  
+  public Set<NodeId> getAssociatedNodeIds() {
+    return new HashSet<NodeId>(nodeIds);
+  }
+
+  public void addNode(Resource nodeRes) {
+    Resources.addTo(resource, nodeRes);
+    numActiveNMs++;
+  }
+  
+  public void removeNode(Resource nodeRes) {
+    Resources.subtractFrom(resource, nodeRes);
+    numActiveNMs--;
+  }
+
+  public Resource getResource() {
+    return this.resource;
+  }
+
+  public int getNumActiveNMs() {
+    return numActiveNMs;
+  }
+  
+  public String getLabelName() {
+    return labelName;
+  }
+  
+  public void setIsExclusive(boolean exclusive) {
+    this.exclusive = exclusive;
+  }
+  
+  public boolean getIsExclusive() {
+    return this.exclusive;
+  }
+  
+  public RMNodeLabel getCopy() {
+    return new RMNodeLabel(labelName, resource, numActiveNMs);
+  }
+  
+  @Override
+  public int compareTo(RMNodeLabel o) {
+    // We should always put empty label entry first after sorting
+    if (labelName.isEmpty() != o.getLabelName().isEmpty()) {
+      if (labelName.isEmpty()) {
+        return -1;
+      }
+      return 1;
+    }
+    
+    return labelName.compareTo(o.getLabelName());
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    if (obj instanceof RMNodeLabel) {
+      RMNodeLabel other = (RMNodeLabel) obj;
+      return Resources.equals(resource, other.getResource())
+          && StringUtils.equals(labelName, other.getLabelName())
+          && (other.getNumActiveNMs() == numActiveNMs); 
+    }
+    return false;
+  }
+  
+  @Override
+  public int hashCode() {
+    final int prime = 502357;
+    return (int) ((((long) labelName.hashCode() << 8)
+        + (resource.hashCode() << 4) + numActiveNMs) % prime);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/event/NodeLabelsStoreEventType.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/event/NodeLabelsStoreEventType.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/event/NodeLabelsStoreEventType.java
index efa2dbe..ce29176 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/event/NodeLabelsStoreEventType.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/event/NodeLabelsStoreEventType.java
@@ -21,5 +21,6 @@ package org.apache.hadoop.yarn.nodelabels.event;
 public enum NodeLabelsStoreEventType {
   REMOVE_LABELS,
   ADD_LABELS,
-  STORE_NODE_TO_LABELS
+  STORE_NODE_TO_LABELS,
+  UPDATE_NODE_LABELS
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/event/StoreUpdateNodeLabelsEvent.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/event/StoreUpdateNodeLabelsEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/event/StoreUpdateNodeLabelsEvent.java
new file mode 100644
index 0000000..7c05012
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/nodelabels/event/StoreUpdateNodeLabelsEvent.java
@@ -0,0 +1,36 @@
+/**
+ * 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.yarn.nodelabels.event;
+
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+
+public class StoreUpdateNodeLabelsEvent extends NodeLabelsStoreEvent {
+  private List<NodeLabel> updatedNodeLabels;
+
+  public StoreUpdateNodeLabelsEvent(List<NodeLabel> updateNodeLabels) {
+    super(NodeLabelsStoreEventType.UPDATE_NODE_LABELS);
+    this.updatedNodeLabels = updateNodeLabels;
+  }
+
+  public List<NodeLabel> getUpdatedNodeLabels() {
+    return updatedNodeLabels;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
index c2d813b..8cb225f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/client/ResourceManagerAdministrationProtocolPBClientImpl.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Refre
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshUserToGroupsMappingsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RemoveFromClusterNodeLabelsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeLabelsRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceRequestProto;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocolPB;
@@ -61,6 +62,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLa
 import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl;
@@ -81,6 +84,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RemoveFromClust
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RemoveFromClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeLabelsRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceResponsePBImpl;
 
@@ -263,4 +268,18 @@ public class ResourceManagerAdministrationProtocolPBClientImpl implements Resour
       return null;
     }
   }
+
+  @Override
+  public UpdateNodeLabelsResponse updateNodeLabels(
+      UpdateNodeLabelsRequest request) throws YarnException, IOException {
+    UpdateNodeLabelsRequestProto requestProto =
+        ((UpdateNodeLabelsRequestPBImpl) request).getProto();
+    try {
+      return new UpdateNodeLabelsResponsePBImpl(
+          proxy.updateNodeLabels(null, requestProto));
+    } catch (ServiceException e) {
+      RPCUtil.unwrapAndThrowException(e);
+      return null;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
index 0eaf581..fe4c812 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/impl/pb/service/ResourceManagerAdministrationProtocolPBServiceImpl.java
@@ -42,6 +42,8 @@ import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.Remov
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RemoveFromClusterNodeLabelsResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeLabelsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeLabelsResponseProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceResponseProto;
 import org.apache.hadoop.yarn.server.api.ResourceManagerAdministrationProtocol;
@@ -55,6 +57,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshSuperUserGroupsC
 import org.apache.hadoop.yarn.server.api.protocolrecords.RefreshUserToGroupsMappingsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl;
@@ -74,6 +78,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RemoveFromClust
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RemoveFromClusterNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeLabelsRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeLabelsResponsePBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceRequestPBImpl;
 import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceResponsePBImpl;
 
@@ -268,4 +274,21 @@ public class ResourceManagerAdministrationProtocolPBServiceImpl implements Resou
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public UpdateNodeLabelsResponseProto updateNodeLabels(
+      RpcController controller, UpdateNodeLabelsRequestProto proto)
+      throws ServiceException {
+    UpdateNodeLabelsRequest request =
+        new UpdateNodeLabelsRequestPBImpl(proto);
+    try {
+      UpdateNodeLabelsResponse response =
+          real.updateNodeLabels(request);
+      return ((UpdateNodeLabelsResponsePBImpl) response).getProto();
+    } catch (YarnException e) {
+      throw new ServiceException(e);
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeLabelsRequestPBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeLabelsRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeLabelsRequestPBImpl.java
new file mode 100644
index 0000000..f18a8db
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeLabelsRequestPBImpl.java
@@ -0,0 +1,145 @@
+/**
+ * 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.yarn.server.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeLabelsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeLabelsRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeLabelsRequest;
+
+public class UpdateNodeLabelsRequestPBImpl extends
+    UpdateNodeLabelsRequest {
+  UpdateNodeLabelsRequestProto proto =
+      UpdateNodeLabelsRequestProto.getDefaultInstance();
+  UpdateNodeLabelsRequestProto.Builder builder = null;
+  private List<NodeLabel> updatedNodeLabels;
+  boolean viaProto = false;
+
+  public UpdateNodeLabelsRequestPBImpl() {
+    builder = UpdateNodeLabelsRequestProto.newBuilder();
+  }
+
+  public UpdateNodeLabelsRequestPBImpl(
+      UpdateNodeLabelsRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public UpdateNodeLabelsRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.updatedNodeLabels != null) {
+      addNodeLabelsToProto();
+    }
+  }
+
+  private void addNodeLabelsToProto() {
+    maybeInitBuilder();
+    builder.clearNodeLabels();
+    List<NodeLabelProto> protoList =
+        new ArrayList<NodeLabelProto>();
+    for (NodeLabel r : this.updatedNodeLabels) {
+      protoList.add(convertToProtoFormat(r));
+    }
+    builder.addAllNodeLabels(protoList);
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    assert false : "hashCode not designed";
+    return 0;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = UpdateNodeLabelsRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public void setNodeLabels(List<NodeLabel> updatedNodeLabels) {
+    maybeInitBuilder();
+    if (updatedNodeLabels == null) {
+      builder.clearNodeLabels();
+    }
+    this.updatedNodeLabels = updatedNodeLabels;
+  }
+
+  private void initLocalNodeLabels() {
+    UpdateNodeLabelsRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<NodeLabelProto> attributesProtoList =
+        p.getNodeLabelsList();
+    this.updatedNodeLabels = new ArrayList<NodeLabel>();
+    for (NodeLabelProto r : attributesProtoList) {
+      this.updatedNodeLabels.add(convertFromProtoFormat(r));
+    }
+  }
+
+  @Override
+  public List<NodeLabel> getNodeLabels() {
+    if (this.updatedNodeLabels != null) {
+      return this.updatedNodeLabels;
+    }
+    initLocalNodeLabels();
+    return this.updatedNodeLabels;
+  }
+
+  private NodeLabel
+      convertFromProtoFormat(NodeLabelProto p) {
+    return new NodeLabelPBImpl(p);
+  }
+
+  private NodeLabelProto convertToProtoFormat(NodeLabel t) {
+    return ((NodeLabelPBImpl) t).getProto();
+  }
+
+  @Override
+  public String toString() {
+    return getProto().toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeLabelsResponsePBImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeLabelsResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeLabelsResponsePBImpl.java
new file mode 100644
index 0000000..217b8fc
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeLabelsResponsePBImpl.java
@@ -0,0 +1,67 @@
+/**
+ * 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.yarn.server.api.protocolrecords.impl.pb;
+
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeLabelsResponseProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeLabelsResponse;
+
+public class UpdateNodeLabelsResponsePBImpl extends
+    UpdateNodeLabelsResponse {
+  UpdateNodeLabelsResponseProto proto =
+      UpdateNodeLabelsResponseProto.getDefaultInstance();
+  UpdateNodeLabelsResponseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  public UpdateNodeLabelsResponsePBImpl() {
+    builder = UpdateNodeLabelsResponseProto.newBuilder();
+  }
+
+  public UpdateNodeLabelsResponsePBImpl(
+      UpdateNodeLabelsResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public UpdateNodeLabelsResponseProto getProto() {
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return getProto().toString().replaceAll("\\n", ", ")
+        .replaceAll("\\s+", " ");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
index 8b48798..80299c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestPBImplRecords.java
@@ -36,15 +36,280 @@ import java.util.Set;
 import org.apache.commons.lang.math.LongRange;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.security.proto.SecurityProtos.*;
-import org.apache.hadoop.yarn.api.protocolrecords.*;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.*;
-import org.apache.hadoop.yarn.api.records.*;
-import org.apache.hadoop.yarn.api.records.impl.pb.*;
-import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.*;
-import org.apache.hadoop.yarn.proto.YarnProtos.*;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.*;
-import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.*;
+import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
+import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto;
+import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
+import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenResponseProto;
+import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
+import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenResponseProto;
+import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.AllocateResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.CancelDelegationTokenResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.FinishApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptReportRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptReportResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptsRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationAttemptsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationReportResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetApplicationsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterMetricsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeLabelsRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodeLabelsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetClusterNodesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerReportRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerReportResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainersRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainersResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetDelegationTokenRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetDelegationTokenResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetLabelsToNodesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetLabelsToNodesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNewApplicationResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToLabelsRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetNodesToLabelsResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueInfoRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueInfoResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetQueueUserAclsInfoResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.KillApplicationResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQueuesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.MoveApplicationAcrossQueuesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RenewDelegationTokenResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationDeleteRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationDeleteResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationSubmissionRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationSubmissionResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationUpdateRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.ReservationUpdateResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.SubmitApplicationResponsePBImpl;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ApplicationReport;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
+import org.apache.hadoop.yarn.api.records.Container;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.ContainerReport;
+import org.apache.hadoop.yarn.api.records.ContainerResourceDecrease;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncrease;
+import org.apache.hadoop.yarn.api.records.ContainerResourceIncreaseRequest;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.LogAggregationContext;
+import org.apache.hadoop.yarn.api.records.NMToken;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
+import org.apache.hadoop.yarn.api.records.NodeReport;
+import org.apache.hadoop.yarn.api.records.PreemptionContainer;
+import org.apache.hadoop.yarn.api.records.PreemptionContract;
+import org.apache.hadoop.yarn.api.records.PreemptionMessage;
+import org.apache.hadoop.yarn.api.records.PreemptionResourceRequest;
+import org.apache.hadoop.yarn.api.records.Priority;
+import org.apache.hadoop.yarn.api.records.QueueInfo;
+import org.apache.hadoop.yarn.api.records.QueueState;
+import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
+import org.apache.hadoop.yarn.api.records.ReservationDefinition;
+import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.ReservationRequest;
+import org.apache.hadoop.yarn.api.records.ReservationRequests;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
+import org.apache.hadoop.yarn.api.records.ResourceOption;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
+import org.apache.hadoop.yarn.api.records.Token;
+import org.apache.hadoop.yarn.api.records.URL;
+import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptReportPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationResourceUsageReportPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerLaunchContextPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerReportPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceDecreasePBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreasePBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerResourceIncreaseRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.NodeReportPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.PreemptionContainerPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.PreemptionContractPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.PreemptionMessagePBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.PreemptionResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.PriorityPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.QueueInfoPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.QueueUserACLInfoPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ResourceBlacklistRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ResourceOptionPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ResourceRequestPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.StrictPreemptionContractPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.TokenPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.URLPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.YarnClusterMetricsPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationAttemptReportProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationReportProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationResourceUsageReportProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerReportProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceDecreaseProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerResourceIncreaseRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeLabelProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.NodeReportProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContainerProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionContractProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionMessageProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PreemptionResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.QueueInfoProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.QueueUserACLInfoProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceBlacklistRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceOptionProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.StrictPreemptionContractProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.URLProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.YarnClusterMetricsProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.AddToClusterNodeLabelsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshAdminAclsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshNodesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshQueuesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshServiceAclsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshSuperUserGroupsConfigurationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshSuperUserGroupsConfigurationResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshUserToGroupsMappingsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RefreshUserToGroupsMappingsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RemoveFromClusterNodeLabelsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.RemoveFromClusterNodeLabelsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.ReplaceLabelsOnNodeResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeLabelsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeLabelsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServerResourceManagerServiceProtos.UpdateNodeResourceResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.AllocateResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.FinishApplicationMasterResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptReportResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationAttemptsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationReportResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetApplicationsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterMetricsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterMetricsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodeLabelsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodeLabelsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetClusterNodesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerReportRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerReportResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainersRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainersResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetLabelsToNodesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetLabelsToNodesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewApplicationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNewApplicationResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNodesToLabelsRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetNodesToLabelsResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueInfoResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetQueueUserAclsInfoResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.KillApplicationResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.MoveApplicationAcrossQueuesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.NMTokenProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationDeleteResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationSubmissionResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ReservationUpdateResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.SubmitApplicationResponseProto;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.AddToClusterNodeLabelsResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshAdminAclsResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshNodesResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshQueuesRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshQueuesResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshServiceAclsRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshServiceAclsResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshSuperUserGroupsConfigurationResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RefreshUserToGroupsMappingsResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RemoveFromClusterNodeLabelsRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.RemoveFromClusterNodeLabelsResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.ReplaceLabelsOnNodeResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeLabelsRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeLabelsResponsePBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceRequestPBImpl;
+import org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb.UpdateNodeResourceResponsePBImpl;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -212,6 +477,7 @@ public class TestPBImplRecords {
     generateByNewInstance(StrictPreemptionContract.class);
     generateByNewInstance(PreemptionMessage.class);
     generateByNewInstance(StartContainerRequest.class);
+    generateByNewInstance(NodeLabel.class);
     // genByNewInstance does not apply to QueueInfo, cause
     // it is recursive(has sub queues)
     typeValueCache.put(QueueInfo.class, QueueInfo.newInstance("root", 1.0f,
@@ -1015,4 +1281,22 @@ public class TestPBImplRecords {
     validatePBImplRecord(GetLabelsToNodesResponsePBImpl.class,
         GetLabelsToNodesResponseProto.class);
   }
+  
+  @Test
+  public void testNodeLabelAttributesPBImpl() throws Exception {
+    validatePBImplRecord(NodeLabelPBImpl.class,
+        NodeLabelProto.class);
+  }
+  
+  @Test
+  public void testUpdateNodeLabelsRequestPBImpl() throws Exception {
+    validatePBImplRecord(UpdateNodeLabelsRequestPBImpl.class,
+        UpdateNodeLabelsRequestProto.class);
+  }
+  
+  @Test
+  public void testUpdateNodeLabelsResponsePBImpl() throws Exception {
+    validatePBImplRecord(UpdateNodeLabelsResponsePBImpl.class,
+        UpdateNodeLabelsResponseProto.class);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
index 65ea79f..67e6119 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/DummyCommonNodeLabelsManager.java
@@ -20,17 +20,20 @@ package org.apache.hadoop.yarn.nodelabels;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.event.InlineDispatcher;
 
 public class DummyCommonNodeLabelsManager extends CommonNodeLabelsManager {
   Map<NodeId, Set<String>> lastNodeToLabels = null;
   Collection<String> lastAddedlabels = null;
   Collection<String> lastRemovedlabels = null;
+  List<NodeLabel> lastUpdatedNodeLabels = null;
 
   @Override
   public void initNodeLabelStore(Configuration conf) {
@@ -58,6 +61,12 @@ public class DummyCommonNodeLabelsManager extends CommonNodeLabelsManager {
       }
 
       @Override
+      public void updateNodeLabels(List<NodeLabel> updatedNodeLabels)
+        throws IOException {
+        lastUpdatedNodeLabels = updatedNodeLabels;
+      }
+
+      @Override
       public void close() throws IOException {
         // do nothing 
       }


[42/51] [abbrv] hadoop git commit: HDFS-7713. Implement mkdirs in the HDFS Web UI. Contributed by Ravi Prakash.

Posted by ka...@apache.org.
HDFS-7713. Implement mkdirs in the HDFS Web UI. Contributed by Ravi Prakash.


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

Branch: refs/heads/YARN-2139
Commit: e38ef70fbc60f062992c834b1cca6e9ba4baef6e
Parents: a16bfff
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Mar 24 15:48:52 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Mar 24 15:48:52 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 +
 .../src/main/webapps/hdfs/explorer.html         | 53 ++++++++++++++++++--
 .../src/main/webapps/hdfs/explorer.js           | 22 ++++++++
 3 files changed, 72 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e38ef70f/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 4ec0891..5ade5fb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -335,6 +335,8 @@ Release 2.8.0 - UNRELEASED
     HDFS-7854. Separate class DataStreamer out of DFSOutputStream. (Li Bo via
     jing9)
 
+    HDFS-7713. Implement mkdirs in the HDFS Web UI. (Ravi Prakash via wheat9)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e38ef70f/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
index 7b34044..cd6623c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html
@@ -87,13 +87,56 @@
 	<button type="button" class="close" onclick="$('#alert-panel').hide();">&times;</button>
 	<div class="alert-body" id="alert-panel-body"></div>
       </div>
+
+    <div class="modal" id="btn-create-directory" tabindex="-1" role="dialog"
+      aria-hidden="true">
+      <div class="modal-dialog">
+        <div class="modal-content">
+          <div class="modal-header">
+            <button type="button" class="close"
+              data-dismiss="modal" aria-hidden="true">&times;</button>
+            <h4 class="modal-title">Create Directory</h4>
+          </div>
+          <div class="modal-body">
+            <div class="form-group">
+              <div class="input-group">
+                <span class="input-group-addon" id="new_directory_pwd"></span>
+                <input type="text" class="form-control" id="new_directory"
+                  placeholder="New Directory Name" />
+              </div>
+            </div>
+          </div>
+          <div class="modal-footer">
+            <button type="button" class="btn" data-dismiss="modal">Cancel</button>
+            <button type="button" class="btn btn-success"
+              id="btn-create-directory-send" data-complete-text="Creating...">
+              Create
+            </button>
+          </div>
+        </div>
+      </div>
+    </div>
+
       <div class="row">
-	<form onsubmit="return false;">
-	  <div class="input-group"><input type="text" class="form-control" id=
-					  "directory" /> <span class="input-group-btn"><button class="btn btn-default"
-											       type="submit" id="btn-nav-directory"><span class="input-group-btn">Go!</span></button></span></div>
-	</form>
+      <div class="col-xs-11">
+        <form onsubmit="return false;">
+          <div class="input-group">
+            <input type="text" class="form-control" id="directory"/>
+            <span class="input-group-btn">
+              <button class="btn btn-default" type="button" id="btn-nav-directory">Go!</button>
+            </span>
+          </div>
+        </form>
+      </div>
+      <div class="col-xs-1">
+        <button type="button" class="btn btn-default" data-toggle="modal"
+          aria-label="New Directory" data-target="#btn-create-directory"
+          title="Create Directory">
+            <span class="glyphicon glyphicon-folder-open"></span>
+        </button>
       </div>
+    </div>
+
       <br />
       <div id="panel"></div>
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e38ef70f/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
index 131b2aa..5572880 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js
@@ -193,5 +193,27 @@
     }
   }
 
+  $('#btn-create-directory').on('show.bs.modal', function(event) {
+    var modal = $(this)
+    $('#new_directory_pwd').html(current_directory);
+  });
+
+  $('#btn-create-directory-send').click(function () {
+    $(this).prop('disabled', true);
+    $(this).button('complete');
+
+    var url = '/webhdfs/v1' + encode_path(append_path(current_directory,
+      $('#new_directory').val())) + '?op=MKDIRS';
+
+    $.ajax(url, { type: 'PUT' }
+    ).done(function(data) {
+      browse_directory(current_directory);
+    }).error(network_error_handler(url)
+     ).complete(function() {
+       $('#btn-create-directory').modal('hide');
+       $('#btn-create-directory-send').button('reset');
+    });
+  })
+
   init();
 })();


[04/51] [abbrv] hadoop git commit: YARN-3269. Yarn.nodemanager.remote-app-log-dir could not be configured to fully qualified path. Contributed by Xuan Gong

Posted by ka...@apache.org.
YARN-3269. Yarn.nodemanager.remote-app-log-dir could not be configured to fully qualified path. Contributed by Xuan Gong


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

Branch: refs/heads/YARN-2139
Commit: d81109e588493cef31e68508a3d671203bd23e12
Parents: d4f7e25
Author: Junping Du <ju...@apache.org>
Authored: Fri Mar 20 13:41:22 2015 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Fri Mar 20 13:41:22 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                 | 3 +++
 .../apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java  | 5 +++--
 .../containermanager/logaggregation/AppLogAggregatorImpl.java   | 2 +-
 .../containermanager/logaggregation/LogAggregationService.java  | 2 +-
 .../logaggregation/TestLogAggregationService.java               | 4 +++-
 5 files changed, 11 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d81109e5/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 00b2c19..bbd018a 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -83,6 +83,9 @@ Release 2.8.0 - UNRELEASED
 
     YARN-3351. AppMaster tracking URL is broken in HA. (Anubhav Dhoot via kasha)
 
+    YARN-3269. Yarn.nodemanager.remote-app-log-dir could not be configured to 
+    fully qualified path. (Xuan Gong via junping_du)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d81109e5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
index ad2ee50..57f655b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogFormat.java
@@ -379,7 +379,7 @@ public class AggregatedLogFormat {
             userUgi.doAs(new PrivilegedExceptionAction<FSDataOutputStream>() {
               @Override
               public FSDataOutputStream run() throws Exception {
-                fc = FileContext.getFileContext(conf);
+                fc = FileContext.getFileContext(remoteAppLogFile.toUri(), conf);
                 fc.setUMask(APP_LOG_FILE_UMASK);
                 return fc.create(
                     remoteAppLogFile,
@@ -471,7 +471,8 @@ public class AggregatedLogFormat {
 
     public LogReader(Configuration conf, Path remoteAppLogFile)
         throws IOException {
-      FileContext fileContext = FileContext.getFileContext(conf);
+      FileContext fileContext =
+          FileContext.getFileContext(remoteAppLogFile.toUri(), conf);
       this.fsDataIStream = fileContext.open(remoteAppLogFile);
       reader =
           new TFile.Reader(this.fsDataIStream, fileContext.getFileStatus(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d81109e5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
index ff70a68..393576b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
@@ -303,7 +303,7 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
         userUgi.doAs(new PrivilegedExceptionAction<Object>() {
           @Override
           public Object run() throws Exception {
-            FileSystem remoteFS = FileSystem.get(conf);
+            FileSystem remoteFS = remoteNodeLogFileForApp.getFileSystem(conf);
             if (remoteFS.exists(remoteNodeTmpLogFileForApp)) {
               if (rename) {
                 remoteFS.rename(remoteNodeTmpLogFileForApp, renamedPath);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d81109e5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java
index bd3e847..0018d56 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/LogAggregationService.java
@@ -176,7 +176,7 @@ public class LogAggregationService extends AbstractService implements
   }
 
   protected FileSystem getFileSystem(Configuration conf) throws IOException {
-    return FileSystem.get(conf);
+    return this.remoteRootLogDir.getFileSystem(conf);
   }
 
   void verifyAndCreateRemoteLogDir(Configuration conf) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d81109e5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
index df51a0d..9cbf153 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
@@ -1299,8 +1299,10 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     logAggregationContextWithInterval.setRolledLogsIncludePattern(".*");
     logAggregationContextWithInterval.setRolledLogsExcludePattern("std_final");
     this.conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
+    //configure YarnConfiguration.NM_REMOTE_APP_LOG_DIR to
+    //have fully qualified path
     this.conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
-      this.remoteRootLogDir.getAbsolutePath());
+      "file://" + this.remoteRootLogDir.getAbsolutePath());
     this.conf.setLong(
       YarnConfiguration.NM_LOG_AGGREGATION_ROLL_MONITORING_INTERVAL_SECONDS,
       3600);


[50/51] [abbrv] hadoop git commit: YARN-2213. Change proxy-user cookie log in AmIpFilter to DEBUG. Contributed by Varun Saxena

Posted by ka...@apache.org.
YARN-2213. Change proxy-user cookie log in AmIpFilter to DEBUG.
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/e556198e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e556198e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e556198e

Branch: refs/heads/YARN-2139
Commit: e556198e71df6be3a83e5598265cb702fc7a668b
Parents: c770df4
Author: Xuan <xg...@apache.org>
Authored: Wed Mar 25 04:49:43 2015 -0700
Committer: Xuan <xg...@apache.org>
Committed: Wed Mar 25 04:49:43 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                | 3 +++
 .../hadoop/yarn/server/webproxy/amfilter/AmIpFilter.java       | 6 ++++--
 2 files changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e556198e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 6a495d9..42b4662 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -837,6 +837,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3393. Getting application(s) goes wrong when app finishes before
     starting the attempt. (Zhijie Shen via xgong)
 
+    YARN-2213. Change proxy-user cookie log in AmIpFilter to DEBUG.
+    (Varun Saxena via xgong)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e556198e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/amfilter/AmIpFilter.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/amfilter/AmIpFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/amfilter/AmIpFilter.java
index f1a8be6..e7617f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/amfilter/AmIpFilter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/amfilter/AmIpFilter.java
@@ -152,8 +152,10 @@ public class AmIpFilter implements Filter {
       }
     }
     if (user == null) {
-      LOG.warn("Could not find " + WebAppProxyServlet.PROXY_USER_COOKIE_NAME
-               + " cookie, so user will not be set");
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Could not find " + WebAppProxyServlet.PROXY_USER_COOKIE_NAME
+                 + " cookie, so user will not be set");
+      }
       chain.doFilter(req, resp);
     } else {
       final AmIpPrincipal principal = new AmIpPrincipal(user);


[40/51] [abbrv] hadoop git commit: HDFS-7854. Separate class DataStreamer out of DFSOutputStream. Contributed by Li Bo.

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/a16bfff7/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
new file mode 100644
index 0000000..6047825
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -0,0 +1,1754 @@
+/**
+ * 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;
+
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.hdfs.util.ByteArrayManager;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
+import org.apache.htrace.NullScope;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceInfo;
+import org.apache.htrace.TraceScope;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+
+/*********************************************************************
+ *
+ * The DataStreamer class is responsible for sending data packets to the
+ * datanodes in the pipeline. It retrieves a new blockid and block locations
+ * from the namenode, and starts streaming packets to the pipeline of
+ * Datanodes. Every packet has a sequence number associated with
+ * it. When all the packets for a block are sent out and acks for each
+ * if them are received, the DataStreamer closes the current block.
+ *
+ * The DataStreamer thread picks up packets from the dataQueue, sends it to
+ * the first datanode in the pipeline and moves it from the dataQueue to the
+ * ackQueue. The ResponseProcessor receives acks from the datanodes. When an
+ * successful ack for a packet is received from all datanodes, the
+ * ResponseProcessor removes the corresponding packet from the ackQueue.
+ *
+ * In case of error, all outstanding packets are moved from ackQueue. A new
+ * pipeline is setup by eliminating the bad datanode from the original
+ * pipeline. The DataStreamer now starts sending packets from the dataQueue.
+ *
+ *********************************************************************/
+
+class DataStreamer extends Daemon {
+  /**
+   * Create a socket for a write pipeline
+   *
+   * @param first the first datanode
+   * @param length the pipeline length
+   * @param client client
+   * @return the socket connected to the first datanode
+   */
+  static Socket createSocketForPipeline(final DatanodeInfo first,
+      final int length, final DFSClient client) throws IOException {
+    final String dnAddr = first.getXferAddr(
+        client.getConf().connectToDnViaHostname);
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
+    }
+    final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
+    final Socket sock = client.socketFactory.createSocket();
+    final int timeout = client.getDatanodeReadTimeout(length);
+    NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), client.getConf().socketTimeout);
+    sock.setSoTimeout(timeout);
+    sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
+    if(DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("Send buf size " + sock.getSendBufferSize());
+    }
+    return sock;
+  }
+
+  /**
+   * if this file is lazy persist
+   *
+   * @param stat the HdfsFileStatus of a file
+   * @return if this file is lazy persist
+   */
+  static boolean isLazyPersist(HdfsFileStatus stat) {
+    final BlockStoragePolicy p = blockStoragePolicySuite.getPolicy(
+        HdfsConstants.MEMORY_STORAGE_POLICY_NAME);
+    return p != null && stat.getStoragePolicy() == p.getId();
+  }
+
+  /**
+   * release a list of packets to ByteArrayManager
+   *
+   * @param packets packets to be release
+   * @param bam ByteArrayManager
+   */
+  private static void releaseBuffer(List<DFSPacket> packets, ByteArrayManager bam) {
+    for(DFSPacket p : packets) {
+      p.releaseBuffer(bam);
+    }
+    packets.clear();
+  }
+
+  private volatile boolean streamerClosed = false;
+  private ExtendedBlock block; // its length is number of bytes acked
+  private Token<BlockTokenIdentifier> accessToken;
+  private DataOutputStream blockStream;
+  private DataInputStream blockReplyStream;
+  private ResponseProcessor response = null;
+  private volatile DatanodeInfo[] nodes = null; // list of targets for current block
+  private volatile StorageType[] storageTypes = null;
+  private volatile String[] storageIDs = null;
+  private String[] favoredNodes;
+  volatile boolean hasError = false;
+  volatile int errorIndex = -1;
+  // Restarting node index
+  AtomicInteger restartingNodeIndex = new AtomicInteger(-1);
+  private long restartDeadline = 0; // Deadline of DN restart
+  private BlockConstructionStage stage;  // block construction stage
+  private long bytesSent = 0; // number of bytes that've been sent
+  private final boolean isLazyPersistFile;
+
+  /** Nodes have been used in the pipeline before and have failed. */
+  private final List<DatanodeInfo> failed = new ArrayList<>();
+  /** The last ack sequence number before pipeline failure. */
+  private long lastAckedSeqnoBeforeFailure = -1;
+  private int pipelineRecoveryCount = 0;
+  /** Has the current block been hflushed? */
+  private boolean isHflushed = false;
+  /** Append on an existing block? */
+  private boolean isAppend;
+
+  private long currentSeqno = 0;
+  private long lastQueuedSeqno = -1;
+  private long lastAckedSeqno = -1;
+  private long bytesCurBlock = 0; // bytes written in current block
+  private final AtomicReference<IOException> lastException = new AtomicReference<>();
+  private Socket s;
+
+  private final DFSClient dfsClient;
+  private final String src;
+  /** Only for DataTransferProtocol.writeBlock(..) */
+  private final DataChecksum checksum4WriteBlock;
+  private final Progressable progress;
+  private final HdfsFileStatus stat;
+  // appending to existing partial block
+  private volatile boolean appendChunk = false;
+  // both dataQueue and ackQueue are protected by dataQueue lock
+  private final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
+  private final LinkedList<DFSPacket> ackQueue = new LinkedList<>();
+  private final AtomicReference<CachingStrategy> cachingStrategy;
+  private final ByteArrayManager byteArrayManager;
+  private static final BlockStoragePolicySuite blockStoragePolicySuite =
+      BlockStoragePolicySuite.createDefaultSuite();
+  //persist blocks on namenode
+  private final AtomicBoolean persistBlocks = new AtomicBoolean(false);
+  private boolean failPacket = false;
+  private final long dfsclientSlowLogThresholdMs;
+  private long artificialSlowdown = 0;
+
+  private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
+
+  private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src,
+                       Progressable progress, DataChecksum checksum,
+                       AtomicReference<CachingStrategy> cachingStrategy,
+                       ByteArrayManager byteArrayManage){
+    this.dfsClient = dfsClient;
+    this.src = src;
+    this.progress = progress;
+    this.stat = stat;
+    this.checksum4WriteBlock = checksum;
+    this.cachingStrategy = cachingStrategy;
+    this.byteArrayManager = byteArrayManage;
+    isLazyPersistFile = isLazyPersist(stat);
+    this.dfsclientSlowLogThresholdMs =
+        dfsClient.getConf().dfsclientSlowIoWarningThresholdMs;
+    excludedNodes = initExcludedNodes();
+  }
+
+  /**
+   * construction with tracing info
+   */
+  DataStreamer(HdfsFileStatus stat, ExtendedBlock block, DFSClient dfsClient,
+               String src, Progressable progress, DataChecksum checksum,
+               AtomicReference<CachingStrategy> cachingStrategy,
+               ByteArrayManager byteArrayManage) {
+    this(stat, dfsClient, src, progress, checksum, cachingStrategy,
+        byteArrayManage);
+    isAppend = false;
+    this.block = block;
+    stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
+  }
+
+  /**
+   * Construct a data streamer for appending to the last partial block
+   * @param lastBlock last block of the file to be appended
+   * @param stat status of the file to be appended
+   * @throws IOException if error occurs
+   */
+  DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
+               String src, Progressable progress, DataChecksum checksum,
+               AtomicReference<CachingStrategy> cachingStrategy,
+               ByteArrayManager byteArrayManage) throws IOException {
+    this(stat, dfsClient, src, progress, checksum, cachingStrategy,
+        byteArrayManage);
+    isAppend = true;
+    stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
+    block = lastBlock.getBlock();
+    bytesSent = block.getNumBytes();
+    accessToken = lastBlock.getBlockToken();
+  }
+
+  /**
+   * Set pipeline in construction
+   *
+   * @param lastBlock the last block of a file
+   * @throws IOException
+   */
+  void setPipelineInConstruction(LocatedBlock lastBlock) throws IOException{
+    // setup pipeline to append to the last block XXX retries??
+    setPipeline(lastBlock);
+    errorIndex = -1;   // no errors yet.
+    if (nodes.length < 1) {
+      throw new IOException("Unable to retrieve blocks locations " +
+          " for last block " + block +
+          "of file " + src);
+    }
+  }
+
+  private void setPipeline(LocatedBlock lb) {
+    setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
+  }
+
+  private void setPipeline(DatanodeInfo[] nodes, StorageType[] storageTypes,
+                           String[] storageIDs) {
+    this.nodes = nodes;
+    this.storageTypes = storageTypes;
+    this.storageIDs = storageIDs;
+  }
+
+  /**
+   * Set favored nodes
+   *
+   * @param favoredNodes favored nodes
+   */
+  void setFavoredNodes(String[] favoredNodes) {
+    this.favoredNodes = favoredNodes;
+  }
+
+  /**
+   * Initialize for data streaming
+   */
+  private void initDataStreaming() {
+    this.setName("DataStreamer for file " + src +
+        " block " + block);
+    response = new ResponseProcessor(nodes);
+    response.start();
+    stage = BlockConstructionStage.DATA_STREAMING;
+  }
+
+  private void endBlock() {
+    if(DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("Closing old block " + block);
+    }
+    this.setName("DataStreamer for file " + src);
+    closeResponder();
+    closeStream();
+    setPipeline(null, null, null);
+    stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
+  }
+
+  /*
+   * streamer thread is the only thread that opens streams to datanode,
+   * and closes them. Any error recovery is also done by this thread.
+   */
+  @Override
+  public void run() {
+    long lastPacket = Time.monotonicNow();
+    TraceScope scope = NullScope.INSTANCE;
+    while (!streamerClosed && dfsClient.clientRunning) {
+      // if the Responder encountered an error, shutdown Responder
+      if (hasError && response != null) {
+        try {
+          response.close();
+          response.join();
+          response = null;
+        } catch (InterruptedException  e) {
+          DFSClient.LOG.warn("Caught exception ", e);
+        }
+      }
+
+      DFSPacket one;
+      try {
+        // process datanode IO errors if any
+        boolean doSleep = false;
+        if (hasError && (errorIndex >= 0 || restartingNodeIndex.get() >= 0)) {
+          doSleep = processDatanodeError();
+        }
+
+        synchronized (dataQueue) {
+          // wait for a packet to be sent.
+          long now = Time.monotonicNow();
+          while ((!streamerClosed && !hasError && dfsClient.clientRunning
+              && dataQueue.size() == 0 &&
+              (stage != BlockConstructionStage.DATA_STREAMING ||
+                  stage == BlockConstructionStage.DATA_STREAMING &&
+                      now - lastPacket < dfsClient.getConf().socketTimeout/2)) || doSleep ) {
+            long timeout = dfsClient.getConf().socketTimeout/2 - (now-lastPacket);
+            timeout = timeout <= 0 ? 1000 : timeout;
+            timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
+                timeout : 1000;
+            try {
+              dataQueue.wait(timeout);
+            } catch (InterruptedException  e) {
+              DFSClient.LOG.warn("Caught exception ", e);
+            }
+            doSleep = false;
+            now = Time.monotonicNow();
+          }
+          if (streamerClosed || hasError || !dfsClient.clientRunning) {
+            continue;
+          }
+          // get packet to be sent.
+          if (dataQueue.isEmpty()) {
+            one = createHeartbeatPacket();
+            assert one != null;
+          } else {
+            one = dataQueue.getFirst(); // regular data packet
+            long parents[] = one.getTraceParents();
+            if (parents.length > 0) {
+              scope = Trace.startSpan("dataStreamer", new TraceInfo(0, parents[0]));
+              // TODO: use setParents API once it's available from HTrace 3.2
+              // scope = Trace.startSpan("dataStreamer", Sampler.ALWAYS);
+              // scope.getSpan().setParents(parents);
+            }
+          }
+        }
+
+        // get new block from namenode.
+        if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
+          if(DFSClient.LOG.isDebugEnabled()) {
+            DFSClient.LOG.debug("Allocating new block");
+          }
+          setPipeline(nextBlockOutputStream());
+          initDataStreaming();
+        } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
+          if(DFSClient.LOG.isDebugEnabled()) {
+            DFSClient.LOG.debug("Append to block " + block);
+          }
+          setupPipelineForAppendOrRecovery();
+          initDataStreaming();
+        }
+
+        long lastByteOffsetInBlock = one.getLastByteOffsetBlock();
+        if (lastByteOffsetInBlock > stat.getBlockSize()) {
+          throw new IOException("BlockSize " + stat.getBlockSize() +
+              " is smaller than data size. " +
+              " Offset of packet in block " +
+              lastByteOffsetInBlock +
+              " Aborting file " + src);
+        }
+
+        if (one.isLastPacketInBlock()) {
+          // wait for all data packets have been successfully acked
+          synchronized (dataQueue) {
+            while (!streamerClosed && !hasError &&
+                ackQueue.size() != 0 && dfsClient.clientRunning) {
+              try {
+                // wait for acks to arrive from datanodes
+                dataQueue.wait(1000);
+              } catch (InterruptedException  e) {
+                DFSClient.LOG.warn("Caught exception ", e);
+              }
+            }
+          }
+          if (streamerClosed || hasError || !dfsClient.clientRunning) {
+            continue;
+          }
+          stage = BlockConstructionStage.PIPELINE_CLOSE;
+        }
+
+        // send the packet
+        Span span = null;
+        synchronized (dataQueue) {
+          // move packet from dataQueue to ackQueue
+          if (!one.isHeartbeatPacket()) {
+            span = scope.detach();
+            one.setTraceSpan(span);
+            dataQueue.removeFirst();
+            ackQueue.addLast(one);
+            dataQueue.notifyAll();
+          }
+        }
+
+        if (DFSClient.LOG.isDebugEnabled()) {
+          DFSClient.LOG.debug("DataStreamer block " + block +
+              " sending packet " + one);
+        }
+
+        // write out data to remote datanode
+        TraceScope writeScope = Trace.startSpan("writeTo", span);
+        try {
+          one.writeTo(blockStream);
+          blockStream.flush();
+        } catch (IOException e) {
+          // HDFS-3398 treat primary DN is down since client is unable to
+          // write to primary DN. If a failed or restarting node has already
+          // been recorded by the responder, the following call will have no
+          // effect. Pipeline recovery can handle only one node error at a
+          // time. If the primary node fails again during the recovery, it
+          // will be taken out then.
+          tryMarkPrimaryDatanodeFailed();
+          throw e;
+        } finally {
+          writeScope.close();
+        }
+        lastPacket = Time.monotonicNow();
+
+        // update bytesSent
+        long tmpBytesSent = one.getLastByteOffsetBlock();
+        if (bytesSent < tmpBytesSent) {
+          bytesSent = tmpBytesSent;
+        }
+
+        if (streamerClosed || hasError || !dfsClient.clientRunning) {
+          continue;
+        }
+
+        // Is this block full?
+        if (one.isLastPacketInBlock()) {
+          // wait for the close packet has been acked
+          synchronized (dataQueue) {
+            while (!streamerClosed && !hasError &&
+                ackQueue.size() != 0 && dfsClient.clientRunning) {
+              dataQueue.wait(1000);// wait for acks to arrive from datanodes
+            }
+          }
+          if (streamerClosed || hasError || !dfsClient.clientRunning) {
+            continue;
+          }
+
+          endBlock();
+        }
+        if (progress != null) { progress.progress(); }
+
+        // This is used by unit test to trigger race conditions.
+        if (artificialSlowdown != 0 && dfsClient.clientRunning) {
+          Thread.sleep(artificialSlowdown);
+        }
+      } catch (Throwable e) {
+        // Log warning if there was a real error.
+        if (restartingNodeIndex.get() == -1) {
+          // Since their messages are descriptive enough, do not always
+          // log a verbose stack-trace WARN for quota exceptions.
+          if (e instanceof QuotaExceededException) {
+            DFSClient.LOG.debug("DataStreamer Quota Exception", e);
+          } else {
+            DFSClient.LOG.warn("DataStreamer Exception", e);
+          }
+        }
+        if (e instanceof IOException) {
+          setLastException((IOException)e);
+        } else {
+          setLastException(new IOException("DataStreamer Exception: ",e));
+        }
+        hasError = true;
+        if (errorIndex == -1 && restartingNodeIndex.get() == -1) {
+          // Not a datanode issue
+          streamerClosed = true;
+        }
+      } finally {
+        scope.close();
+      }
+    }
+    closeInternal();
+  }
+
+  private void closeInternal() {
+    closeResponder();       // close and join
+    closeStream();
+    streamerClosed = true;
+    release();
+    synchronized (dataQueue) {
+      dataQueue.notifyAll();
+    }
+  }
+
+  /**
+   * release the DFSPackets in the two queues
+   *
+   */
+  void release() {
+    synchronized (dataQueue) {
+      releaseBuffer(dataQueue, byteArrayManager);
+      releaseBuffer(ackQueue, byteArrayManager);
+    }
+  }
+
+  /**
+   * wait for the ack of seqno
+   *
+   * @param seqno the sequence number to be acked
+   * @throws IOException
+   */
+  void waitForAckedSeqno(long seqno) throws IOException {
+    TraceScope scope = Trace.startSpan("waitForAckedSeqno", Sampler.NEVER);
+    try {
+      if (DFSClient.LOG.isDebugEnabled()) {
+        DFSClient.LOG.debug("Waiting for ack for: " + seqno);
+      }
+      long begin = Time.monotonicNow();
+      try {
+        synchronized (dataQueue) {
+          while (!streamerClosed) {
+            checkClosed();
+            if (lastAckedSeqno >= seqno) {
+              break;
+            }
+            try {
+              dataQueue.wait(1000); // when we receive an ack, we notify on
+              // dataQueue
+            } catch (InterruptedException ie) {
+              throw new InterruptedIOException(
+                  "Interrupted while waiting for data to be acknowledged by pipeline");
+            }
+          }
+        }
+        checkClosed();
+      } catch (ClosedChannelException e) {
+      }
+      long duration = Time.monotonicNow() - begin;
+      if (duration > dfsclientSlowLogThresholdMs) {
+        DFSClient.LOG.warn("Slow waitForAckedSeqno took " + duration
+            + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
+      }
+    } finally {
+      scope.close();
+    }
+  }
+
+  /**
+   * wait for space of dataQueue and queue the packet
+   *
+   * @param packet  the DFSPacket to be queued
+   * @throws IOException
+   */
+  void waitAndQueuePacket(DFSPacket packet) throws IOException {
+    synchronized (dataQueue) {
+      try {
+        // If queue is full, then wait till we have enough space
+        boolean firstWait = true;
+        try {
+          while (!streamerClosed && dataQueue.size() + ackQueue.size() >
+              dfsClient.getConf().writeMaxPackets) {
+            if (firstWait) {
+              Span span = Trace.currentSpan();
+              if (span != null) {
+                span.addTimelineAnnotation("dataQueue.wait");
+              }
+              firstWait = false;
+            }
+            try {
+              dataQueue.wait();
+            } catch (InterruptedException e) {
+              // If we get interrupted while waiting to queue data, we still need to get rid
+              // of the current packet. This is because we have an invariant that if
+              // currentPacket gets full, it will get queued before the next writeChunk.
+              //
+              // Rather than wait around for space in the queue, we should instead try to
+              // return to the caller as soon as possible, even though we slightly overrun
+              // the MAX_PACKETS length.
+              Thread.currentThread().interrupt();
+              break;
+            }
+          }
+        } finally {
+          Span span = Trace.currentSpan();
+          if ((span != null) && (!firstWait)) {
+            span.addTimelineAnnotation("end.wait");
+          }
+        }
+        checkClosed();
+        queuePacket(packet);
+      } catch (ClosedChannelException e) {
+      }
+    }
+  }
+
+  /*
+   * close the streamer, should be called only by an external thread
+   * and only after all data to be sent has been flushed to datanode.
+   *
+   * Interrupt this data streamer if force is true
+   *
+   * @param force if this data stream is forced to be closed
+   */
+  void close(boolean force) {
+    streamerClosed = true;
+    synchronized (dataQueue) {
+      dataQueue.notifyAll();
+    }
+    if (force) {
+      this.interrupt();
+    }
+  }
+
+
+  private void checkClosed() throws IOException {
+    if (streamerClosed) {
+      IOException e = lastException.get();
+      throw e != null ? e : new ClosedChannelException();
+    }
+  }
+
+  private void closeResponder() {
+    if (response != null) {
+      try {
+        response.close();
+        response.join();
+      } catch (InterruptedException  e) {
+        DFSClient.LOG.warn("Caught exception ", e);
+      } finally {
+        response = null;
+      }
+    }
+  }
+
+  private void closeStream() {
+    if (blockStream != null) {
+      try {
+        blockStream.close();
+      } catch (IOException e) {
+        setLastException(e);
+      } finally {
+        blockStream = null;
+      }
+    }
+    if (blockReplyStream != null) {
+      try {
+        blockReplyStream.close();
+      } catch (IOException e) {
+        setLastException(e);
+      } finally {
+        blockReplyStream = null;
+      }
+    }
+    if (null != s) {
+      try {
+        s.close();
+      } catch (IOException e) {
+        setLastException(e);
+      } finally {
+        s = null;
+      }
+    }
+  }
+
+  // The following synchronized methods are used whenever
+  // errorIndex or restartingNodeIndex is set. This is because
+  // check & set needs to be atomic. Simply reading variables
+  // does not require a synchronization. When responder is
+  // not running (e.g. during pipeline recovery), there is no
+  // need to use these methods.
+
+  /** Set the error node index. Called by responder */
+  synchronized void setErrorIndex(int idx) {
+    errorIndex = idx;
+  }
+
+  /** Set the restarting node index. Called by responder */
+  synchronized void setRestartingNodeIndex(int idx) {
+    restartingNodeIndex.set(idx);
+    // If the data streamer has already set the primary node
+    // bad, clear it. It is likely that the write failed due to
+    // the DN shutdown. Even if it was a real failure, the pipeline
+    // recovery will take care of it.
+    errorIndex = -1;
+  }
+
+  /**
+   * This method is used when no explicit error report was received,
+   * but something failed. When the primary node is a suspect or
+   * unsure about the cause, the primary node is marked as failed.
+   */
+  synchronized void tryMarkPrimaryDatanodeFailed() {
+    // There should be no existing error and no ongoing restart.
+    if ((errorIndex == -1) && (restartingNodeIndex.get() == -1)) {
+      errorIndex = 0;
+    }
+  }
+
+  /**
+   * Examine whether it is worth waiting for a node to restart.
+   * @param index the node index
+   */
+  boolean shouldWaitForRestart(int index) {
+    // Only one node in the pipeline.
+    if (nodes.length == 1) {
+      return true;
+    }
+
+    // Is it a local node?
+    InetAddress addr = null;
+    try {
+      addr = InetAddress.getByName(nodes[index].getIpAddr());
+    } catch (java.net.UnknownHostException e) {
+      // we are passing an ip address. this should not happen.
+      assert false;
+    }
+
+    if (addr != null && NetUtils.isLocalAddress(addr)) {
+      return true;
+    }
+    return false;
+  }
+
+  //
+  // Processes responses from the datanodes.  A packet is removed
+  // from the ackQueue when its response arrives.
+  //
+  private class ResponseProcessor extends Daemon {
+
+    private volatile boolean responderClosed = false;
+    private DatanodeInfo[] targets = null;
+    private boolean isLastPacketInBlock = false;
+
+    ResponseProcessor (DatanodeInfo[] targets) {
+      this.targets = targets;
+    }
+
+    @Override
+    public void run() {
+
+      setName("ResponseProcessor for block " + block);
+      PipelineAck ack = new PipelineAck();
+
+      TraceScope scope = NullScope.INSTANCE;
+      while (!responderClosed && dfsClient.clientRunning && !isLastPacketInBlock) {
+        // process responses from datanodes.
+        try {
+          // read an ack from the pipeline
+          long begin = Time.monotonicNow();
+          ack.readFields(blockReplyStream);
+          long duration = Time.monotonicNow() - begin;
+          if (duration > dfsclientSlowLogThresholdMs
+              && ack.getSeqno() != DFSPacket.HEART_BEAT_SEQNO) {
+            DFSClient.LOG
+                .warn("Slow ReadProcessor read fields took " + duration
+                    + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
+                    + ack + ", targets: " + Arrays.asList(targets));
+          } else if (DFSClient.LOG.isDebugEnabled()) {
+            DFSClient.LOG.debug("DFSClient " + ack);
+          }
+
+          long seqno = ack.getSeqno();
+          // processes response status from datanodes.
+          for (int i = ack.getNumOfReplies()-1; i >=0  && dfsClient.clientRunning; i--) {
+            final Status reply = PipelineAck.getStatusFromHeader(ack
+                .getReply(i));
+            // Restart will not be treated differently unless it is
+            // the local node or the only one in the pipeline.
+            if (PipelineAck.isRestartOOBStatus(reply) &&
+                shouldWaitForRestart(i)) {
+              restartDeadline = dfsClient.getConf().datanodeRestartTimeout
+                  + Time.monotonicNow();
+              setRestartingNodeIndex(i);
+              String message = "A datanode is restarting: " + targets[i];
+              DFSClient.LOG.info(message);
+              throw new IOException(message);
+            }
+            // node error
+            if (reply != SUCCESS) {
+              setErrorIndex(i); // first bad datanode
+              throw new IOException("Bad response " + reply +
+                  " for block " + block +
+                  " from datanode " +
+                  targets[i]);
+            }
+          }
+
+          assert seqno != PipelineAck.UNKOWN_SEQNO :
+              "Ack for unknown seqno should be a failed ack: " + ack;
+          if (seqno == DFSPacket.HEART_BEAT_SEQNO) {  // a heartbeat ack
+            continue;
+          }
+
+          // a success ack for a data packet
+          DFSPacket one;
+          synchronized (dataQueue) {
+            one = ackQueue.getFirst();
+          }
+          if (one.getSeqno() != seqno) {
+            throw new IOException("ResponseProcessor: Expecting seqno " +
+                " for block " + block +
+                one.getSeqno() + " but received " + seqno);
+          }
+          isLastPacketInBlock = one.isLastPacketInBlock();
+
+          // Fail the packet write for testing in order to force a
+          // pipeline recovery.
+          if (DFSClientFaultInjector.get().failPacket() &&
+              isLastPacketInBlock) {
+            failPacket = true;
+            throw new IOException(
+                "Failing the last packet for testing.");
+          }
+
+          // update bytesAcked
+          block.setNumBytes(one.getLastByteOffsetBlock());
+
+          synchronized (dataQueue) {
+            scope = Trace.continueSpan(one.getTraceSpan());
+            one.setTraceSpan(null);
+            lastAckedSeqno = seqno;
+            ackQueue.removeFirst();
+            dataQueue.notifyAll();
+
+            one.releaseBuffer(byteArrayManager);
+          }
+        } catch (Exception e) {
+          if (!responderClosed) {
+            if (e instanceof IOException) {
+              setLastException((IOException)e);
+            }
+            hasError = true;
+            // If no explicit error report was received, mark the primary
+            // node as failed.
+            tryMarkPrimaryDatanodeFailed();
+            synchronized (dataQueue) {
+              dataQueue.notifyAll();
+            }
+            if (restartingNodeIndex.get() == -1) {
+              DFSClient.LOG.warn("DataStreamer ResponseProcessor exception "
+                  + " for block " + block, e);
+            }
+            responderClosed = true;
+          }
+        } finally {
+            scope.close();
+        }
+      }
+    }
+
+    void close() {
+      responderClosed = true;
+      this.interrupt();
+    }
+  }
+
+  // If this stream has encountered any errors so far, shutdown
+  // threads and mark stream as closed. Returns true if we should
+  // sleep for a while after returning from this call.
+  //
+  private boolean processDatanodeError() throws IOException {
+    if (response != null) {
+      DFSClient.LOG.info("Error Recovery for " + block +
+          " waiting for responder to exit. ");
+      return true;
+    }
+    closeStream();
+
+    // move packets from ack queue to front of the data queue
+    synchronized (dataQueue) {
+      dataQueue.addAll(0, ackQueue);
+      ackQueue.clear();
+    }
+
+    // Record the new pipeline failure recovery.
+    if (lastAckedSeqnoBeforeFailure != lastAckedSeqno) {
+      lastAckedSeqnoBeforeFailure = lastAckedSeqno;
+      pipelineRecoveryCount = 1;
+    } else {
+      // If we had to recover the pipeline five times in a row for the
+      // same packet, this client likely has corrupt data or corrupting
+      // during transmission.
+      if (++pipelineRecoveryCount > 5) {
+        DFSClient.LOG.warn("Error recovering pipeline for writing " +
+            block + ". Already retried 5 times for the same packet.");
+        lastException.set(new IOException("Failing write. Tried pipeline " +
+            "recovery 5 times without success."));
+        streamerClosed = true;
+        return false;
+      }
+    }
+    boolean doSleep = setupPipelineForAppendOrRecovery();
+
+    if (!streamerClosed && dfsClient.clientRunning) {
+      if (stage == BlockConstructionStage.PIPELINE_CLOSE) {
+
+        // If we had an error while closing the pipeline, we go through a fast-path
+        // where the BlockReceiver does not run. Instead, the DataNode just finalizes
+        // the block immediately during the 'connect ack' process. So, we want to pull
+        // the end-of-block packet from the dataQueue, since we don't actually have
+        // a true pipeline to send it over.
+        //
+        // We also need to set lastAckedSeqno to the end-of-block Packet's seqno, so that
+        // a client waiting on close() will be aware that the flush finished.
+        synchronized (dataQueue) {
+          DFSPacket endOfBlockPacket = dataQueue.remove();  // remove the end of block packet
+          Span span = endOfBlockPacket.getTraceSpan();
+          if (span != null) {
+            // Close any trace span associated with this Packet
+            TraceScope scope = Trace.continueSpan(span);
+            scope.close();
+          }
+          assert endOfBlockPacket.isLastPacketInBlock();
+          assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1;
+          lastAckedSeqno = endOfBlockPacket.getSeqno();
+          dataQueue.notifyAll();
+        }
+        endBlock();
+      } else {
+        initDataStreaming();
+      }
+    }
+
+    return doSleep;
+  }
+
+  void setHflush() {
+    isHflushed = true;
+  }
+
+  private int findNewDatanode(final DatanodeInfo[] original
+  ) throws IOException {
+    if (nodes.length != original.length + 1) {
+      throw new IOException(
+          new StringBuilder()
+              .append("Failed to replace a bad datanode on the existing pipeline ")
+              .append("due to no more good datanodes being available to try. ")
+              .append("(Nodes: current=").append(Arrays.asList(nodes))
+              .append(", original=").append(Arrays.asList(original)).append("). ")
+              .append("The current failed datanode replacement policy is ")
+              .append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
+              .append("a client may configure this via '")
+              .append(DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_POLICY_KEY)
+              .append("' in its configuration.")
+              .toString());
+    }
+    for(int i = 0; i < nodes.length; i++) {
+      int j = 0;
+      for(; j < original.length && !nodes[i].equals(original[j]); j++);
+      if (j == original.length) {
+        return i;
+      }
+    }
+    throw new IOException("Failed: new datanode not found: nodes="
+        + Arrays.asList(nodes) + ", original=" + Arrays.asList(original));
+  }
+
+  private void addDatanode2ExistingPipeline() throws IOException {
+    if (DataTransferProtocol.LOG.isDebugEnabled()) {
+      DataTransferProtocol.LOG.debug("lastAckedSeqno = " + lastAckedSeqno);
+    }
+      /*
+       * Is data transfer necessary?  We have the following cases.
+       *
+       * Case 1: Failure in Pipeline Setup
+       * - Append
+       *    + Transfer the stored replica, which may be a RBW or a finalized.
+       * - Create
+       *    + If no data, then no transfer is required.
+       *    + If there are data written, transfer RBW. This case may happens
+       *      when there are streaming failure earlier in this pipeline.
+       *
+       * Case 2: Failure in Streaming
+       * - Append/Create:
+       *    + transfer RBW
+       *
+       * Case 3: Failure in Close
+       * - Append/Create:
+       *    + no transfer, let NameNode replicates the block.
+       */
+    if (!isAppend && lastAckedSeqno < 0
+        && stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
+      //no data have been written
+      return;
+    } else if (stage == BlockConstructionStage.PIPELINE_CLOSE
+        || stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
+      //pipeline is closing
+      return;
+    }
+
+    //get a new datanode
+    final DatanodeInfo[] original = nodes;
+    final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode(
+        src, stat.getFileId(), block, nodes, storageIDs,
+        failed.toArray(new DatanodeInfo[failed.size()]),
+        1, dfsClient.clientName);
+    setPipeline(lb);
+
+    //find the new datanode
+    final int d = findNewDatanode(original);
+
+    //transfer replica
+    final DatanodeInfo src = d == 0? nodes[1]: nodes[d - 1];
+    final DatanodeInfo[] targets = {nodes[d]};
+    final StorageType[] targetStorageTypes = {storageTypes[d]};
+    transfer(src, targets, targetStorageTypes, lb.getBlockToken());
+  }
+
+  private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
+                        final StorageType[] targetStorageTypes,
+                        final Token<BlockTokenIdentifier> blockToken) throws IOException {
+    //transfer replica to the new datanode
+    Socket sock = null;
+    DataOutputStream out = null;
+    DataInputStream in = null;
+    try {
+      sock = createSocketForPipeline(src, 2, dfsClient);
+      final long writeTimeout = dfsClient.getDatanodeWriteTimeout(2);
+
+      OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
+      InputStream unbufIn = NetUtils.getInputStream(sock);
+      IOStreamPair saslStreams = dfsClient.saslClient.socketSend(sock,
+          unbufOut, unbufIn, dfsClient, blockToken, src);
+      unbufOut = saslStreams.out;
+      unbufIn = saslStreams.in;
+      out = new DataOutputStream(new BufferedOutputStream(unbufOut,
+          HdfsConstants.SMALL_BUFFER_SIZE));
+      in = new DataInputStream(unbufIn);
+
+      //send the TRANSFER_BLOCK request
+      new Sender(out).transferBlock(block, blockToken, dfsClient.clientName,
+          targets, targetStorageTypes);
+      out.flush();
+
+      //ack
+      BlockOpResponseProto response =
+          BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
+      if (SUCCESS != response.getStatus()) {
+        throw new IOException("Failed to add a datanode");
+      }
+    } finally {
+      IOUtils.closeStream(in);
+      IOUtils.closeStream(out);
+      IOUtils.closeSocket(sock);
+    }
+  }
+
+  /**
+   * Open a DataStreamer to a DataNode pipeline so that
+   * it can be written to.
+   * This happens when a file is appended or data streaming fails
+   * It keeps on trying until a pipeline is setup
+   */
+  private boolean setupPipelineForAppendOrRecovery() throws IOException {
+    // check number of datanodes
+    if (nodes == null || nodes.length == 0) {
+      String msg = "Could not get block locations. " + "Source file \""
+          + src + "\" - Aborting...";
+      DFSClient.LOG.warn(msg);
+      setLastException(new IOException(msg));
+      streamerClosed = true;
+      return false;
+    }
+
+    boolean success = false;
+    long newGS = 0L;
+    while (!success && !streamerClosed && dfsClient.clientRunning) {
+      // Sleep before reconnect if a dn is restarting.
+      // This process will be repeated until the deadline or the datanode
+      // starts back up.
+      if (restartingNodeIndex.get() >= 0) {
+        // 4 seconds or the configured deadline period, whichever is shorter.
+        // This is the retry interval and recovery will be retried in this
+        // interval until timeout or success.
+        long delay = Math.min(dfsClient.getConf().datanodeRestartTimeout,
+            4000L);
+        try {
+          Thread.sleep(delay);
+        } catch (InterruptedException ie) {
+          lastException.set(new IOException("Interrupted while waiting for " +
+              "datanode to restart. " + nodes[restartingNodeIndex.get()]));
+          streamerClosed = true;
+          return false;
+        }
+      }
+      boolean isRecovery = hasError;
+      // remove bad datanode from list of datanodes.
+      // If errorIndex was not set (i.e. appends), then do not remove
+      // any datanodes
+      //
+      if (errorIndex >= 0) {
+        StringBuilder pipelineMsg = new StringBuilder();
+        for (int j = 0; j < nodes.length; j++) {
+          pipelineMsg.append(nodes[j]);
+          if (j < nodes.length - 1) {
+            pipelineMsg.append(", ");
+          }
+        }
+        if (nodes.length <= 1) {
+          lastException.set(new IOException("All datanodes " + pipelineMsg
+              + " are bad. Aborting..."));
+          streamerClosed = true;
+          return false;
+        }
+        DFSClient.LOG.warn("Error Recovery for block " + block +
+            " in pipeline " + pipelineMsg +
+            ": bad datanode " + nodes[errorIndex]);
+        failed.add(nodes[errorIndex]);
+
+        DatanodeInfo[] newnodes = new DatanodeInfo[nodes.length-1];
+        arraycopy(nodes, newnodes, errorIndex);
+
+        final StorageType[] newStorageTypes = new StorageType[newnodes.length];
+        arraycopy(storageTypes, newStorageTypes, errorIndex);
+
+        final String[] newStorageIDs = new String[newnodes.length];
+        arraycopy(storageIDs, newStorageIDs, errorIndex);
+
+        setPipeline(newnodes, newStorageTypes, newStorageIDs);
+
+        // Just took care of a node error while waiting for a node restart
+        if (restartingNodeIndex.get() >= 0) {
+          // If the error came from a node further away than the restarting
+          // node, the restart must have been complete.
+          if (errorIndex > restartingNodeIndex.get()) {
+            restartingNodeIndex.set(-1);
+          } else if (errorIndex < restartingNodeIndex.get()) {
+            // the node index has shifted.
+            restartingNodeIndex.decrementAndGet();
+          } else {
+            // this shouldn't happen...
+            assert false;
+          }
+        }
+
+        if (restartingNodeIndex.get() == -1) {
+          hasError = false;
+        }
+        lastException.set(null);
+        errorIndex = -1;
+      }
+
+      // Check if replace-datanode policy is satisfied.
+      if (dfsClient.dtpReplaceDatanodeOnFailure.satisfy(stat.getReplication(),
+          nodes, isAppend, isHflushed)) {
+        try {
+          addDatanode2ExistingPipeline();
+        } catch(IOException ioe) {
+          if (!dfsClient.dtpReplaceDatanodeOnFailure.isBestEffort()) {
+            throw ioe;
+          }
+          DFSClient.LOG.warn("Failed to replace datanode."
+              + " Continue with the remaining datanodes since "
+              + DFSConfigKeys.DFS_CLIENT_WRITE_REPLACE_DATANODE_ON_FAILURE_BEST_EFFORT_KEY
+              + " is set to true.", ioe);
+        }
+      }
+
+      // get a new generation stamp and an access token
+      LocatedBlock lb = dfsClient.namenode.updateBlockForPipeline(block, dfsClient.clientName);
+      newGS = lb.getBlock().getGenerationStamp();
+      accessToken = lb.getBlockToken();
+
+      // set up the pipeline again with the remaining nodes
+      if (failPacket) { // for testing
+        success = createBlockOutputStream(nodes, storageTypes, newGS, isRecovery);
+        failPacket = false;
+        try {
+          // Give DNs time to send in bad reports. In real situations,
+          // good reports should follow bad ones, if client committed
+          // with those nodes.
+          Thread.sleep(2000);
+        } catch (InterruptedException ie) {}
+      } else {
+        success = createBlockOutputStream(nodes, storageTypes, newGS, isRecovery);
+      }
+
+      if (restartingNodeIndex.get() >= 0) {
+        assert hasError == true;
+        // check errorIndex set above
+        if (errorIndex == restartingNodeIndex.get()) {
+          // ignore, if came from the restarting node
+          errorIndex = -1;
+        }
+        // still within the deadline
+        if (Time.monotonicNow() < restartDeadline) {
+          continue; // with in the deadline
+        }
+        // expired. declare the restarting node dead
+        restartDeadline = 0;
+        int expiredNodeIndex = restartingNodeIndex.get();
+        restartingNodeIndex.set(-1);
+        DFSClient.LOG.warn("Datanode did not restart in time: " +
+            nodes[expiredNodeIndex]);
+        // Mark the restarting node as failed. If there is any other failed
+        // node during the last pipeline construction attempt, it will not be
+        // overwritten/dropped. In this case, the restarting node will get
+        // excluded in the following attempt, if it still does not come up.
+        if (errorIndex == -1) {
+          errorIndex = expiredNodeIndex;
+        }
+        // From this point on, normal pipeline recovery applies.
+      }
+    } // while
+
+    if (success) {
+      // update pipeline at the namenode
+      ExtendedBlock newBlock = new ExtendedBlock(
+          block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
+      dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
+          nodes, storageIDs);
+      // update client side generation stamp
+      block = newBlock;
+    }
+    return false; // do not sleep, continue processing
+  }
+
+  /**
+   * Open a DataStreamer to a DataNode so that it can be written to.
+   * This happens when a file is created and each time a new block is allocated.
+   * Must get block ID and the IDs of the destinations from the namenode.
+   * Returns the list of target datanodes.
+   */
+  private LocatedBlock nextBlockOutputStream() throws IOException {
+    LocatedBlock lb = null;
+    DatanodeInfo[] nodes = null;
+    StorageType[] storageTypes = null;
+    int count = dfsClient.getConf().nBlockWriteRetry;
+    boolean success = false;
+    ExtendedBlock oldBlock = block;
+    do {
+      hasError = false;
+      lastException.set(null);
+      errorIndex = -1;
+      success = false;
+
+      DatanodeInfo[] excluded =
+          excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
+              .keySet()
+              .toArray(new DatanodeInfo[0]);
+      block = oldBlock;
+      lb = locateFollowingBlock(excluded.length > 0 ? excluded : null);
+      block = lb.getBlock();
+      block.setNumBytes(0);
+      bytesSent = 0;
+      accessToken = lb.getBlockToken();
+      nodes = lb.getLocations();
+      storageTypes = lb.getStorageTypes();
+
+      //
+      // Connect to first DataNode in the list.
+      //
+      success = createBlockOutputStream(nodes, storageTypes, 0L, false);
+
+      if (!success) {
+        DFSClient.LOG.info("Abandoning " + block);
+        dfsClient.namenode.abandonBlock(block, stat.getFileId(), src,
+            dfsClient.clientName);
+        block = null;
+        DFSClient.LOG.info("Excluding datanode " + nodes[errorIndex]);
+        excludedNodes.put(nodes[errorIndex], nodes[errorIndex]);
+      }
+    } while (!success && --count >= 0);
+
+    if (!success) {
+      throw new IOException("Unable to create new block.");
+    }
+    return lb;
+  }
+
+  // connects to the first datanode in the pipeline
+  // Returns true if success, otherwise return failure.
+  //
+  private boolean createBlockOutputStream(DatanodeInfo[] nodes,
+      StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
+    if (nodes.length == 0) {
+      DFSClient.LOG.info("nodes are empty for write pipeline of block "
+          + block);
+      return false;
+    }
+    Status pipelineStatus = SUCCESS;
+    String firstBadLink = "";
+    boolean checkRestart = false;
+    if (DFSClient.LOG.isDebugEnabled()) {
+      for (int i = 0; i < nodes.length; i++) {
+        DFSClient.LOG.debug("pipeline = " + nodes[i]);
+      }
+    }
+
+    // persist blocks on namenode on next flush
+    persistBlocks.set(true);
+
+    int refetchEncryptionKey = 1;
+    while (true) {
+      boolean result = false;
+      DataOutputStream out = null;
+      try {
+        assert null == s : "Previous socket unclosed";
+        assert null == blockReplyStream : "Previous blockReplyStream unclosed";
+        s = createSocketForPipeline(nodes[0], nodes.length, dfsClient);
+        long writeTimeout = dfsClient.getDatanodeWriteTimeout(nodes.length);
+
+        OutputStream unbufOut = NetUtils.getOutputStream(s, writeTimeout);
+        InputStream unbufIn = NetUtils.getInputStream(s);
+        IOStreamPair saslStreams = dfsClient.saslClient.socketSend(s,
+            unbufOut, unbufIn, dfsClient, accessToken, nodes[0]);
+        unbufOut = saslStreams.out;
+        unbufIn = saslStreams.in;
+        out = new DataOutputStream(new BufferedOutputStream(unbufOut,
+            HdfsConstants.SMALL_BUFFER_SIZE));
+        blockReplyStream = new DataInputStream(unbufIn);
+
+        //
+        // Xmit header info to datanode
+        //
+
+        BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
+
+        // We cannot change the block length in 'block' as it counts the number
+        // of bytes ack'ed.
+        ExtendedBlock blockCopy = new ExtendedBlock(block);
+        blockCopy.setNumBytes(stat.getBlockSize());
+
+        boolean[] targetPinnings = getPinnings(nodes, true);
+        // send the request
+        new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
+            dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
+            nodes.length, block.getNumBytes(), bytesSent, newGS,
+            checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
+            (targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
+
+        // receive ack for connect
+        BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
+            PBHelper.vintPrefixed(blockReplyStream));
+        pipelineStatus = resp.getStatus();
+        firstBadLink = resp.getFirstBadLink();
+
+        // Got an restart OOB ack.
+        // If a node is already restarting, this status is not likely from
+        // the same node. If it is from a different node, it is not
+        // from the local datanode. Thus it is safe to treat this as a
+        // regular node error.
+        if (PipelineAck.isRestartOOBStatus(pipelineStatus) &&
+            restartingNodeIndex.get() == -1) {
+          checkRestart = true;
+          throw new IOException("A datanode is restarting.");
+        }
+		
+        String logInfo = "ack with firstBadLink as " + firstBadLink;
+        DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
+
+        assert null == blockStream : "Previous blockStream unclosed";
+        blockStream = out;
+        result =  true; // success
+        restartingNodeIndex.set(-1);
+        hasError = false;
+      } catch (IOException ie) {
+        if (restartingNodeIndex.get() == -1) {
+          DFSClient.LOG.info("Exception in createBlockOutputStream", ie);
+        }
+        if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+          DFSClient.LOG.info("Will fetch a new encryption key and retry, "
+              + "encryption key was invalid when connecting to "
+              + nodes[0] + " : " + ie);
+          // The encryption key used is invalid.
+          refetchEncryptionKey--;
+          dfsClient.clearDataEncryptionKey();
+          // Don't close the socket/exclude this node just yet. Try again with
+          // a new encryption key.
+          continue;
+        }
+
+        // find the datanode that matches
+        if (firstBadLink.length() != 0) {
+          for (int i = 0; i < nodes.length; i++) {
+            // NB: Unconditionally using the xfer addr w/o hostname
+            if (firstBadLink.equals(nodes[i].getXferAddr())) {
+              errorIndex = i;
+              break;
+            }
+          }
+        } else {
+          assert checkRestart == false;
+          errorIndex = 0;
+        }
+        // Check whether there is a restart worth waiting for.
+        if (checkRestart && shouldWaitForRestart(errorIndex)) {
+          restartDeadline = dfsClient.getConf().datanodeRestartTimeout
+              + Time.monotonicNow();
+          restartingNodeIndex.set(errorIndex);
+          errorIndex = -1;
+          DFSClient.LOG.info("Waiting for the datanode to be restarted: " +
+              nodes[restartingNodeIndex.get()]);
+        }
+        hasError = true;
+        setLastException(ie);
+        result =  false;  // error
+      } finally {
+        if (!result) {
+          IOUtils.closeSocket(s);
+          s = null;
+          IOUtils.closeStream(out);
+          out = null;
+          IOUtils.closeStream(blockReplyStream);
+          blockReplyStream = null;
+        }
+      }
+      return result;
+    }
+  }
+
+  private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
+    if (favoredNodes == null) {
+      return null;
+    } else {
+      boolean[] pinnings = new boolean[nodes.length];
+      HashSet<String> favoredSet =
+          new HashSet<String>(Arrays.asList(favoredNodes));
+      for (int i = 0; i < nodes.length; i++) {
+        pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
+        if (DFSClient.LOG.isDebugEnabled()) {
+          DFSClient.LOG.debug(nodes[i].getXferAddrWithHostname() +
+              " was chosen by name node (favored=" + pinnings[i] +
+              ").");
+        }
+      }
+      if (shouldLog && !favoredSet.isEmpty()) {
+        // There is one or more favored nodes that were not allocated.
+        DFSClient.LOG.warn(
+            "These favored nodes were specified but not chosen: " +
+                favoredSet +
+                " Specified favored nodes: " + Arrays.toString(favoredNodes));
+
+      }
+      return pinnings;
+    }
+  }
+
+  private LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
+      throws IOException {
+    int retries = dfsClient.getConf().nBlockWriteLocateFollowingRetry;
+    long sleeptime = dfsClient.getConf().
+        blockWriteLocateFollowingInitialDelayMs;
+    while (true) {
+      long localstart = Time.monotonicNow();
+      while (true) {
+        try {
+          return dfsClient.namenode.addBlock(src, dfsClient.clientName,
+              block, excludedNodes, stat.getFileId(), favoredNodes);
+        } catch (RemoteException e) {
+          IOException ue =
+              e.unwrapRemoteException(FileNotFoundException.class,
+                  AccessControlException.class,
+                  NSQuotaExceededException.class,
+                  DSQuotaExceededException.class,
+                  UnresolvedPathException.class);
+          if (ue != e) {
+            throw ue; // no need to retry these exceptions
+          }
+
+
+          if (NotReplicatedYetException.class.getName().
+              equals(e.getClassName())) {
+            if (retries == 0) {
+              throw e;
+            } else {
+              --retries;
+              DFSClient.LOG.info("Exception while adding a block", e);
+              long elapsed = Time.monotonicNow() - localstart;
+              if (elapsed > 5000) {
+                DFSClient.LOG.info("Waiting for replication for "
+                    + (elapsed / 1000) + " seconds");
+              }
+              try {
+                DFSClient.LOG.warn("NotReplicatedYetException sleeping " + src
+                    + " retries left " + retries);
+                Thread.sleep(sleeptime);
+                sleeptime *= 2;
+              } catch (InterruptedException ie) {
+                DFSClient.LOG.warn("Caught exception ", ie);
+              }
+            }
+          } else {
+            throw e;
+          }
+
+        }
+      }
+    }
+  }
+
+  /**
+   * get the block this streamer is writing to
+   *
+   * @return the block this streamer is writing to
+   */
+  ExtendedBlock getBlock() {
+    return block;
+  }
+
+  /**
+   * return the target datanodes in the pipeline
+   *
+   * @return the target datanodes in the pipeline
+   */
+  DatanodeInfo[] getNodes() {
+    return nodes;
+  }
+
+  /**
+   * return the token of the block
+   *
+   * @return the token of the block
+   */
+  Token<BlockTokenIdentifier> getBlockToken() {
+    return accessToken;
+  }
+
+  /**
+   * set last exception
+   *
+   * @param e an exception
+   */
+  void setLastException(IOException e) {
+    lastException.compareAndSet(null, e);
+  }
+
+  /**
+   * Put a packet to the data queue
+   *
+   * @param packet the packet to be put into the data queued
+   */
+  void queuePacket(DFSPacket packet) {
+    synchronized (dataQueue) {
+      if (packet == null) return;
+      packet.addTraceParent(Trace.currentSpan());
+      dataQueue.addLast(packet);
+      lastQueuedSeqno = packet.getSeqno();
+      if (DFSClient.LOG.isDebugEnabled()) {
+        DFSClient.LOG.debug("Queued packet " + packet.getSeqno());
+      }
+      dataQueue.notifyAll();
+    }
+  }
+
+  /**
+   * For heartbeat packets, create buffer directly by new byte[]
+   * since heartbeats should not be blocked.
+   */
+  private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
+    final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
+    return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
+  }
+
+  private LoadingCache<DatanodeInfo, DatanodeInfo> initExcludedNodes() {
+    return CacheBuilder.newBuilder().expireAfterWrite(
+        dfsClient.getConf().excludedNodesCacheExpiry, TimeUnit.MILLISECONDS)
+        .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
+          @Override
+          public void onRemoval(
+              RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
+            DFSClient.LOG.info("Removing node " + notification.getKey()
+                + " from the excluded nodes list");
+          }
+        }).build(new CacheLoader<DatanodeInfo, DatanodeInfo>() {
+          @Override
+          public DatanodeInfo load(DatanodeInfo key) throws Exception {
+            return key;
+          }
+        });
+  }
+
+  private static <T> void arraycopy(T[] srcs, T[] dsts, int skipIndex) {
+    System.arraycopy(srcs, 0, dsts, 0, skipIndex);
+    System.arraycopy(srcs, skipIndex+1, dsts, skipIndex, dsts.length-skipIndex);
+  }
+
+  /**
+   * check if to persist blocks on namenode
+   *
+   * @return if to persist blocks on namenode
+   */
+  AtomicBoolean getPersistBlocks(){
+    return persistBlocks;
+  }
+
+  /**
+   * check if to append a chunk
+   *
+   * @param appendChunk if to append a chunk
+   */
+  void setAppendChunk(boolean appendChunk){
+    this.appendChunk = appendChunk;
+  }
+
+  /**
+   * get if to append a chunk
+   *
+   * @return if to append a chunk
+   */
+  boolean getAppendChunk(){
+    return appendChunk;
+  }
+
+  /**
+   * get the last exception
+   *
+   * @return the last exception
+   */
+  AtomicReference<IOException> getLastException(){
+    return lastException;
+  }
+
+  /**
+   * get the socket connecting to the first datanode in pipeline
+   *
+   * @return socket connecting to the first datanode in pipeline
+   */
+  Socket getSocket() {
+    return s;
+  }
+
+  /**
+   * set socket to null
+   */
+  void setSocketToNull() {
+    this.s = null;
+  }
+
+  /**
+   * return current sequence number and then increase it by 1
+   *
+   * @return current sequence number before increasing
+   */
+  long getAndIncCurrentSeqno() {
+    long old = this.currentSeqno;
+    this.currentSeqno++;
+    return old;
+  }
+
+  /**
+   * get last queued sequence number
+   *
+   * @return last queued sequence number
+   */
+  long getLastQueuedSeqno() {
+    return lastQueuedSeqno;
+  }
+
+  /**
+   * get the number of bytes of current block
+   *
+   * @return the number of bytes of current block
+   */
+  long getBytesCurBlock() {
+    return bytesCurBlock;
+  }
+
+  /**
+   * set the bytes of current block that have been written
+   *
+   * @param bytesCurBlock bytes of current block that have been written
+   */
+  void setBytesCurBlock(long bytesCurBlock) {
+    this.bytesCurBlock = bytesCurBlock;
+  }
+
+  /**
+   * increase bytes of current block by len.
+   *
+   * @param len how many bytes to increase to current block
+   */
+  void incBytesCurBlock(long len) {
+    this.bytesCurBlock += len;
+  }
+
+  /**
+   * set artificial slow down for unit test
+   *
+   * @param period artificial slow down
+   */
+  void setArtificialSlowdown(long period) {
+    this.artificialSlowdown = period;
+  }
+
+  /**
+   * if this streamer is to terminate
+   *
+   * @return if this streamer is to terminate
+   */
+  boolean streamerClosed(){
+    return streamerClosed;
+  }
+
+  void closeSocket() throws IOException {
+    if (s != null) {
+      s.close();
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a16bfff7/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 493351b..5fc78d1 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
@@ -905,7 +905,7 @@ public class DFSTestUtil {
   public static BlockOpResponseProto transferRbw(final ExtendedBlock b, 
       final DFSClient dfsClient, final DatanodeInfo... datanodes) throws IOException {
     assertEquals(2, datanodes.length);
-    final Socket s = DFSOutputStream.createSocketForPipeline(datanodes[0],
+    final Socket s = DataStreamer.createSocketForPipeline(datanodes[0],
         datanodes.length, dfsClient);
     final long writeTimeout = dfsClient.getDatanodeWriteTimeout(datanodes.length);
     final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a16bfff7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
index 7269e39..b47e7f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSOutputStream.java
@@ -51,8 +51,11 @@ public class TestDFSOutputStream {
     DFSOutputStream dos = (DFSOutputStream) Whitebox.getInternalState(os,
         "wrappedStream");
     @SuppressWarnings("unchecked")
+    DataStreamer streamer = (DataStreamer) Whitebox
+        .getInternalState(dos, "streamer");
+    @SuppressWarnings("unchecked")
     AtomicReference<IOException> ex = (AtomicReference<IOException>) Whitebox
-        .getInternalState(dos, "lastException");
+        .getInternalState(streamer, "lastException");
     Assert.assertEquals(null, ex.get());
 
     dos.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a16bfff7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
index e1c547b..fd916a9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java
@@ -43,6 +43,8 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.FileReader;
 import java.io.IOException;
+import java.lang.reflect.Field;
+import java.lang.reflect.Modifier;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.UnknownHostException;
@@ -603,7 +605,8 @@ public class TestFileCreation {
    * Test that file leases are persisted across namenode restarts.
    */
   @Test
-  public void testFileCreationNamenodeRestart() throws IOException {
+  public void testFileCreationNamenodeRestart()
+      throws IOException, NoSuchFieldException, IllegalAccessException {
     Configuration conf = new HdfsConfiguration();
     final int MAX_IDLE_TIME = 2000; // 2s
     conf.setInt("ipc.client.connection.maxidletime", MAX_IDLE_TIME);
@@ -702,11 +705,18 @@ public class TestFileCreation {
       // new blocks for files that were renamed.
       DFSOutputStream dfstream = (DFSOutputStream)
                                                  (stm.getWrappedStream());
-      dfstream.setTestFilename(file1.toString());
+
+      Field f = DFSOutputStream.class.getDeclaredField("src");
+      Field modifiersField = Field.class.getDeclaredField("modifiers");
+      modifiersField.setAccessible(true);
+      modifiersField.setInt(f, f.getModifiers() & ~Modifier.FINAL);
+      f.setAccessible(true);
+
+      f.set(dfstream, file1.toString());
       dfstream = (DFSOutputStream) (stm3.getWrappedStream());
-      dfstream.setTestFilename(file3new.toString());
+      f.set(dfstream, file3new.toString());
       dfstream = (DFSOutputStream) (stm4.getWrappedStream());
-      dfstream.setTestFilename(file4new.toString());
+      f.set(dfstream, file4new.toString());
 
       // write 1 byte to file.  This should succeed because the 
       // namenode should have persisted leases.


[17/51] [abbrv] hadoop git commit: MAPREDUCE-6287. Deprecated methods in org.apache.hadoop.examples.Sort. Contributed by Chao Zhang.

Posted by ka...@apache.org.
MAPREDUCE-6287. Deprecated methods in org.apache.hadoop.examples.Sort. Contributed by Chao Zhang.


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

Branch: refs/heads/YARN-2139
Commit: b375d1fc936913edf4a75212559f160c41043906
Parents: 4cd54d9
Author: Harsh J <ha...@cloudera.com>
Authored: Mon Mar 23 03:48:36 2015 +0530
Committer: Harsh J <ha...@cloudera.com>
Committed: Mon Mar 23 03:48:36 2015 +0530

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                          | 3 +++
 .../src/main/java/org/apache/hadoop/examples/Sort.java        | 7 ++++---
 2 files changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b375d1fc/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index b75d8aa..20505b6 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -256,6 +256,9 @@ Release 2.8.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    MAPREDUCE-6287. Deprecated methods in org.apache.hadoop.examples.Sort
+    (Chao Zhang via harsh)
+
     MAPREDUCE-5190. Unnecessary condition test in RandomSampler.
     (Jingguo Yao via harsh)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b375d1fc/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/Sort.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/Sort.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/Sort.java
index a90c02b..0382c09 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/Sort.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/Sort.java
@@ -24,7 +24,7 @@ import java.util.*;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.mapreduce.filecache.DistributedCache;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Writable;
@@ -160,13 +160,14 @@ public class Sort<K,V> extends Configured implements Tool {
       System.out.println("Sampling input to effect total-order sort...");
       job.setPartitionerClass(TotalOrderPartitioner.class);
       Path inputDir = FileInputFormat.getInputPaths(job)[0];
-      inputDir = inputDir.makeQualified(inputDir.getFileSystem(conf));
+      FileSystem fs = inputDir.getFileSystem(conf);
+      inputDir = inputDir.makeQualified(fs.getUri(), fs.getWorkingDirectory());
       Path partitionFile = new Path(inputDir, "_sortPartitioning");
       TotalOrderPartitioner.setPartitionFile(conf, partitionFile);
       InputSampler.<K,V>writePartitionFile(job, sampler);
       URI partitionUri = new URI(partitionFile.toString() +
                                  "#" + "_sortPartitioning");
-      DistributedCache.addCacheFile(partitionUri, conf);
+      job.addCacheFile(partitionUri);
     }
 
     System.out.println("Running on " +


[08/51] [abbrv] hadoop git commit: YARN-3345. Add non-exclusive node label API. Contributed by Wangda Tan

Posted by ka...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
index d05c75c..1e2326b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestCommonNodeLabelsManager.java
@@ -29,7 +29,9 @@ import java.util.Set;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -536,4 +538,30 @@ public class TestCommonNodeLabelsManager extends NodeLabelTestBase {
     Assert.assertTrue("Should failed when #labels > 1 on a host after add",
         failed);
   }
+
+  @Test (timeout = 5000)
+  public void testUpdateNodeLabels() throws Exception {
+    boolean failed = false;
+
+    // should fail: label isn't exist
+    try {
+      mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance(
+        "p1", false)));
+    } catch (YarnException e) {
+      failed = true;
+    }
+    Assert.assertTrue("Should fail since the node label doesn't exist", failed);
+
+    mgr.addToCluserNodeLabels(toSet("p1", "p2", "p3"));
+
+    mgr.updateNodeLabels(Arrays.asList(
+        NodeLabel.newInstance("p1", false), NodeLabel.newInstance("p2", true)));
+    Assert.assertEquals("p1", mgr.lastUpdatedNodeLabels.get(0).getNodeLabel());
+    Assert.assertFalse(mgr.lastUpdatedNodeLabels.get(0).getIsExclusive());
+    Assert.assertTrue(mgr.lastUpdatedNodeLabels.get(1).getIsExclusive());
+
+    // Check exclusive for p1/p2
+    Assert.assertFalse(mgr.isExclusiveNodeLabel("p1"));
+    Assert.assertTrue(mgr.isExclusiveNodeLabel("p2"));
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
index 5cc026a..6694290 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/nodelabels/TestFileSystemNodeLabelsStore.java
@@ -24,6 +24,7 @@ import java.util.Arrays;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.InlineDispatcher;
 import org.junit.After;
@@ -188,7 +189,7 @@ public class TestFileSystemNodeLabelsStore extends NodeLabelTestBase {
   }
   
   @SuppressWarnings({ "unchecked", "rawtypes" })
-  @Test//(timeout = 10000)
+  @Test (timeout = 10000)
   public void testSerilizationAfterRecovery() throws Exception {
     mgr.addToCluserNodeLabels(toSet("p1", "p2", "p3"));
     mgr.addToCluserNodeLabels(toSet("p4"));
@@ -218,6 +219,14 @@ public class TestFileSystemNodeLabelsStore extends NodeLabelTestBase {
      * p4: n4 
      * p6: n6, n7
      */
+
+    mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("p2", false)));
+    mgr.updateNodeLabels(Arrays.asList(NodeLabel.newInstance("p6", false)));
+
+    /*
+     * Set p2/p6 to be exclusive
+     */
+
     // shutdown mgr and start a new mgr
     mgr.stop();
 
@@ -239,6 +248,10 @@ public class TestFileSystemNodeLabelsStore extends NodeLabelTestBase {
         "p4", toSet(toNodeId("n4")),
         "p2", toSet(toNodeId("n2"))));
 
+    Assert.assertFalse(mgr.isExclusiveNodeLabel("p2"));
+    Assert.assertTrue(mgr.isExclusiveNodeLabel("p4"));
+    Assert.assertFalse(mgr.isExclusiveNodeLabel("p6"));
+
     /*
      * Add label p7,p8 then shutdown
      */

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/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 6180995..22b92c2 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
@@ -77,6 +77,8 @@ import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLa
 import org.apache.hadoop.yarn.server.api.protocolrecords.RemoveFromClusterNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.ReplaceLabelsOnNodeResponse;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeLabelsRequest;
+import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeLabelsResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.UpdateNodeResourceResponse;
 import org.apache.hadoop.yarn.server.resourcemanager.reservation.ReservationSystem;
@@ -663,6 +665,28 @@ public class AdminService extends CompositeService implements
       throw logAndWrapException(ioe, user.getShortUserName(), argName, msg);
     }
   }
+  
+  @Override
+  public UpdateNodeLabelsResponse updateNodeLabels(
+      UpdateNodeLabelsRequest request) throws YarnException, IOException {
+    String argName = "updateNodeLabels";
+    final String msg = "update node labels";
+    UserGroupInformation user = checkAcls(argName);
+
+    checkRMStatus(user.getShortUserName(), argName, msg);
+
+    UpdateNodeLabelsResponse response = UpdateNodeLabelsResponse.newInstance();
+    
+    try {
+      rmContext.getNodeLabelManager().updateNodeLabels(
+          request.getNodeLabels());
+      RMAuditLogger
+      .logSuccess(user.getShortUserName(), argName, "AdminService");
+      return response;
+    } catch (YarnException ioe) {
+      throw logAndWrapException(ioe, user.getShortUserName(), argName, msg);
+    }
+  }
 
   private void checkRMStatus(String user, String argName, String msg)
       throws StandbyException {
@@ -673,11 +697,11 @@ public class AdminService extends CompositeService implements
     }
   }
 
-  private YarnException logAndWrapException(IOException ioe, String user,
+  private YarnException logAndWrapException(Exception exception, String user,
       String argName, String msg) throws YarnException {
-    LOG.info("Exception " + msg, ioe);
+    LOG.info("Exception " + msg, exception);
     RMAuditLogger.logFailure(user, argName, "", 
         "AdminService", "Exception " + msg);
-    return RPCUtil.getRemoteException(ioe);
+    return RPCUtil.getRemoteException(exception);
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.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/nodelabels/RMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
index e5abdc9..574e24c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/RMNodeLabelsManager.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
-import org.apache.hadoop.yarn.nodelabels.NodeLabel;
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.security.YarnAuthorizationProvider;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeLabelsUpdateSchedulerEvent;
@@ -45,7 +45,6 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import com.google.common.collect.ImmutableSet;
 
 public class RMNodeLabelsManager extends CommonNodeLabelsManager {
-  
   protected static class Queue {
     protected Set<String> acccessibleNodeLabels;
     protected Resource resource;
@@ -156,7 +155,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
       throws IOException {
     try {
       writeLock.lock();
-      
+
       // get nodesCollection before edition
       Map<String, Host> before = cloneNodeMap(replaceLabelsToNode.keySet());
 
@@ -171,7 +170,6 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
       writeLock.unlock();
     }
   }
-  
 
   /*
    * Following methods are used for setting if a node is up and running, and it
@@ -201,7 +199,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
       Set<String> labelsForNode = getLabelsByNode(nodeId);
       if (labelsForNode != null) {
         for (String label : labelsForNode) {
-          NodeLabel labelInfo = labelCollections.get(label);
+          RMNodeLabel labelInfo = labelCollections.get(label);
           if(labelInfo != null) {
             labelInfo.addNodeId(nodeId);
           }
@@ -383,7 +381,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         // no label in the past
         if (oldLabels.isEmpty()) {
           // update labels
-          NodeLabel label = labelCollections.get(NO_LABEL);
+          RMNodeLabel label = labelCollections.get(NO_LABEL);
           label.removeNode(oldNM.resource);
 
           // update queues, all queue can access this node
@@ -393,7 +391,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         } else {
           // update labels
           for (String labelName : oldLabels) {
-            NodeLabel label = labelCollections.get(labelName);
+            RMNodeLabel label = labelCollections.get(labelName);
             if (null == label) {
               continue;
             }
@@ -418,7 +416,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         // no label in the past
         if (newLabels.isEmpty()) {
           // update labels
-          NodeLabel label = labelCollections.get(NO_LABEL);
+          RMNodeLabel label = labelCollections.get(NO_LABEL);
           label.addNode(newNM.resource);
 
           // update queues, all queue can access this node
@@ -428,7 +426,7 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
         } else {
           // update labels
           for (String labelName : newLabels) {
-            NodeLabel label = labelCollections.get(labelName);
+            RMNodeLabel label = labelCollections.get(labelName);
             label.addNode(newNM.resource);
           }
 
@@ -499,13 +497,13 @@ public class RMNodeLabelsManager extends CommonNodeLabelsManager {
     this.rmContext = rmContext;
   }
 
-  public List<NodeLabel> pullRMNodeLabelsInfo() {
+  public List<RMNodeLabel> pullRMNodeLabelsInfo() {
     try {
       readLock.lock();
-      List<NodeLabel> infos = new ArrayList<NodeLabel>();
+      List<RMNodeLabel> infos = new ArrayList<RMNodeLabel>();
 
-      for (Entry<String, NodeLabel> entry : labelCollections.entrySet()) {
-        NodeLabel label = entry.getValue();
+      for (Entry<String, RMNodeLabel> entry : labelCollections.entrySet()) {
+        RMNodeLabel label = entry.getValue();
         infos.add(label.getCopy());
       }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.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/webapp/NodeLabelsPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
index 5e8c1ed..7458558 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NodeLabelsPage.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.webapp;
 
 import static org.apache.hadoop.yarn.webapp.view.JQueryUI.DATATABLES_ID;
 
-import org.apache.hadoop.yarn.nodelabels.NodeLabel;
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
 import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager;
 import org.apache.hadoop.yarn.webapp.SubView;
@@ -55,7 +55,7 @@ public class NodeLabelsPage extends RmView {
           tbody();
   
       RMNodeLabelsManager nlm = rm.getRMContext().getNodeLabelManager();
-      for (NodeLabel info : nlm.pullRMNodeLabelsInfo()) {
+      for (RMNodeLabel info : nlm.pullRMNodeLabelsInfo()) {
         TR<TBODY<TABLE<Hamlet>>> row =
             tbody.tr().td(
                 info.getLabelName().isEmpty() ? "<NO_LABEL>" : info

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.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/nodelabels/NullRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
index b1be525..14d96a0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NullRMNodeLabelsManager.java
@@ -20,11 +20,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.nodelabels;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.NodeLabel;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.nodelabels.NodeLabelsStore;
 
@@ -63,6 +65,12 @@ public class NullRMNodeLabelsManager extends RMNodeLabelsManager {
       public void close() throws IOException {
         // do nothing
       }
+
+      @Override
+      public void updateNodeLabels(List<NodeLabel> updatedNodeLabels)
+          throws IOException {
+        // do nothing
+      }
     };
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e1feb4ea/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.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/nodelabels/TestRMNodeLabelsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
index 8a37c24..ace8742 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/TestRMNodeLabelsManager.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager;
-import org.apache.hadoop.yarn.nodelabels.NodeLabel;
+import org.apache.hadoop.yarn.nodelabels.RMNodeLabel;
 import org.apache.hadoop.yarn.nodelabels.NodeLabelTestBase;
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
@@ -476,8 +476,8 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
     
   }
   
-  private void checkNodeLabelInfo(List<NodeLabel> infos, String labelName, int activeNMs, int memory) {
-    for (NodeLabel info : infos) {
+  private void checkNodeLabelInfo(List<RMNodeLabel> infos, String labelName, int activeNMs, int memory) {
+    for (RMNodeLabel info : infos) {
       if (info.getLabelName().equals(labelName)) {
         Assert.assertEquals(activeNMs, info.getNumActiveNMs());
         Assert.assertEquals(memory, info.getResource().getMemory());
@@ -499,7 +499,7 @@ public class TestRMNodeLabelsManager extends NodeLabelTestBase {
         toNodeId("n2"), toSet("x"), toNodeId("n3"), toSet("y")));
     
     // x, y, z and ""
-    List<NodeLabel> infos = mgr.pullRMNodeLabelsInfo();
+    List<RMNodeLabel> infos = mgr.pullRMNodeLabelsInfo();
     Assert.assertEquals(4, infos.size());
     checkNodeLabelInfo(infos, RMNodeLabelsManager.NO_LABEL, 2, 20);
     checkNodeLabelInfo(infos, "x", 2, 20);


[07/51] [abbrv] hadoop git commit: YARN-3350. YARN RackResolver spams logs with messages at info level. Contributed by Wilfred Spiegelenburg

Posted by ka...@apache.org.
YARN-3350. YARN RackResolver spams logs with messages at info level. Contributed by Wilfred Spiegelenburg


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

Branch: refs/heads/YARN-2139
Commit: 7f1e2f996995e1883d9336f720c27621cf1b73b6
Parents: fe5c23b
Author: Junping Du <ju...@apache.org>
Authored: Fri Mar 20 18:21:33 2015 -0700
Committer: Junping Du <ju...@apache.org>
Committed: Fri Mar 20 18:21:33 2015 -0700

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                           |  3 +++
 .../java/org/apache/hadoop/yarn/util/RackResolver.java    | 10 +++++++---
 2 files changed, 10 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f1e2f99/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 046b7b1..177d587 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -68,6 +68,9 @@ Release 2.8.0 - UNRELEASED
     YARN-3356. Capacity Scheduler FiCaSchedulerApp should use ResourceUsage to
     track used-resources-by-label. (Wangda Tan via jianhe)
 
+    YARN-3350. YARN RackResolver spams logs with messages at info level. 
+    (Wilfred Spiegelenburg via junping_du)
+
   OPTIMIZATIONS
 
     YARN-3339. TestDockerContainerExecutor should pull a single image and not

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7f1e2f99/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RackResolver.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RackResolver.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RackResolver.java
index cc2a56c..c44c2cf 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RackResolver.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/RackResolver.java
@@ -102,11 +102,15 @@ public class RackResolver {
     String rName = null;
     if (rNameList == null || rNameList.get(0) == null) {
       rName = NetworkTopology.DEFAULT_RACK;
-      LOG.info("Couldn't resolve " + hostName + ". Falling back to "
-          + NetworkTopology.DEFAULT_RACK);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Couldn't resolve " + hostName + ". Falling back to "
+            + NetworkTopology.DEFAULT_RACK);
+      }
     } else {
       rName = rNameList.get(0);
-      LOG.info("Resolved " + hostName + " to " + rName);
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Resolved " + hostName + " to " + rName);
+      }
     }
     return new NodeBase(hostName, rName);
   }


[19/51] [abbrv] hadoop git commit: YARN-3384. TestLogAggregationService.verifyContainerLogs fails after YARN-2777. Contributed by Naganarasimha G R.

Posted by ka...@apache.org.
YARN-3384. TestLogAggregationService.verifyContainerLogs fails after YARN-2777. Contributed by Naganarasimha G R.


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

Branch: refs/heads/YARN-2139
Commit: 82eda771e05cf2b31788ee1582551e65f1c0f9aa
Parents: 0b9f12c
Author: Tsuyoshi Ozawa <oz...@apache.org>
Authored: Tue Mar 24 00:25:30 2015 +0900
Committer: Tsuyoshi Ozawa <oz...@apache.org>
Committed: Tue Mar 24 00:25:30 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                  | 3 +++
 .../logaggregation/TestLogAggregationService.java                | 4 +++-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/82eda771/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index f8c1a76..e04624e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -816,6 +816,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3369. Missing NullPointer check in AppSchedulingInfo causes RM to die.
     (Brahma Reddy Battula via wangda)
 
+    YARN-3384. TestLogAggregationService.verifyContainerLogs fails after
+    YARN-2777. (Naganarasimha G R via ozawa)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/82eda771/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
index 9cbf153..b1de9cb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
@@ -804,7 +804,9 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         Map<String, String> thisContainerMap = logMap.remove(containerStr);
         Assert.assertEquals(numOfContainerLogs, thisContainerMap.size());
         for (String fileType : logFiles) {
-          String expectedValue = containerStr + " Hello " + fileType + "!";
+          String expectedValue =
+              containerStr + " Hello " + fileType + "!End of LogType:"
+                  + fileType;
           LOG.info("Expected log-content : " + new String(expectedValue));
           String foundValue = thisContainerMap.remove(fileType);
           Assert.assertNotNull(cId + " " + fileType


[44/51] [abbrv] hadoop git commit: HDFS-7985. WebHDFS should be always enabled. Contributed by Li Lu.

Posted by ka...@apache.org.
HDFS-7985. WebHDFS should be always enabled. Contributed by Li Lu.


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

Branch: refs/heads/YARN-2139
Commit: 80278a5f85a91b3e02e700e0b3c0a433c15e0565
Parents: 53a28af
Author: Haohui Mai <wh...@apache.org>
Authored: Tue Mar 24 21:55:56 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Tue Mar 24 21:55:56 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 +
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |  2 -
 .../server/namenode/NameNodeHttpServer.java     | 46 ++++++++++----------
 .../hadoop/hdfs/web/WebHdfsFileSystem.java      |  7 ---
 .../src/main/native/libhdfs/native_mini_dfs.c   | 16 -------
 .../org/apache/hadoop/fs/TestSymlinkHdfs.java   |  1 -
 .../hadoop/hdfs/TestDistributedFileSystem.java  |  1 -
 .../java/org/apache/hadoop/hdfs/TestQuota.java  |  4 +-
 .../hdfs/security/TestDelegationToken.java      |  1 -
 .../TestDelegationTokenForProxyUser.java        |  1 -
 .../hdfs/server/namenode/TestAuditLogs.java     |  1 -
 .../TestNameNodeRespectsBindHostKeys.java       |  1 -
 .../hdfs/web/TestFSMainOperationsWebHdfs.java   |  1 -
 .../hadoop/hdfs/web/TestHttpsFileSystem.java    |  1 -
 .../org/apache/hadoop/hdfs/web/TestWebHDFS.java | 12 -----
 .../hdfs/web/TestWebHdfsFileSystemContract.java |  1 -
 .../hadoop/hdfs/web/TestWebHdfsTokens.java      |  1 -
 .../web/TestWebHdfsWithMultipleNameNodes.java   |  2 -
 .../apache/hadoop/hdfs/web/WebHdfsTestUtil.java |  1 -
 19 files changed, 25 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/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 4bed2ab..8d7a4e1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -18,6 +18,8 @@ Trunk (Unreleased)
     option since it may incorrectly finalize an ongoing rolling upgrade.
     (Kai Sasaki via szetszwo)
 
+    HDFS-7985. WebHDFS should be always enabled. (Li Lu via wheat9)
+
   NEW FEATURES
 
     HDFS-3125. Add JournalService to enable Journal Daemon. (suresh)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/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 b5bbe5f..d714276 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
@@ -227,8 +227,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final int     DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_DEFAULT = 4;
   public static final String  DFS_WEBHDFS_AUTHENTICATION_FILTER_KEY = "dfs.web.authentication.filter";
   public static final String  DFS_WEBHDFS_AUTHENTICATION_FILTER_DEFAULT = AuthFilter.class.getName();
-  public static final String  DFS_WEBHDFS_ENABLED_KEY = "dfs.webhdfs.enabled";
-  public static final boolean DFS_WEBHDFS_ENABLED_DEFAULT = true;
   public static final String  DFS_WEBHDFS_USER_PATTERN_KEY = "dfs.webhdfs.user.provider.user.pattern";
   public static final String  DFS_WEBHDFS_USER_PATTERN_DEFAULT = "^[A-Za-z_][A-Za-z0-9._-]*[$]?$";
   public static final String  DFS_PERMISSIONS_ENABLED_KEY = "dfs.permissions.enabled";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
index 662c0e9..a671d21 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeHttpServer.java
@@ -67,30 +67,28 @@ public class NameNodeHttpServer {
   }
 
   private void initWebHdfs(Configuration conf) throws IOException {
-    if (WebHdfsFileSystem.isEnabled(conf, HttpServer2.LOG)) {
-      // set user pattern based on configuration file
-      UserParam.setUserPattern(conf.get(
-          DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY,
-          DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT));
-
-      // add authentication filter for webhdfs
-      final String className = conf.get(
-          DFSConfigKeys.DFS_WEBHDFS_AUTHENTICATION_FILTER_KEY,
-          DFSConfigKeys.DFS_WEBHDFS_AUTHENTICATION_FILTER_DEFAULT);
-      final String name = className;
-
-      final String pathSpec = WebHdfsFileSystem.PATH_PREFIX + "/*";
-      Map<String, String> params = getAuthFilterParams(conf);
-      HttpServer2.defineFilter(httpServer.getWebAppContext(), name, className,
-          params, new String[] { pathSpec });
-      HttpServer2.LOG.info("Added filter '" + name + "' (class=" + className
-          + ")");
-
-      // add webhdfs packages
-      httpServer.addJerseyResourcePackage(NamenodeWebHdfsMethods.class
-          .getPackage().getName() + ";" + Param.class.getPackage().getName(),
-          pathSpec);
-    }
+    // set user pattern based on configuration file
+    UserParam.setUserPattern(conf.get(
+        DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY,
+        DFSConfigKeys.DFS_WEBHDFS_USER_PATTERN_DEFAULT));
+
+    // add authentication filter for webhdfs
+    final String className = conf.get(
+        DFSConfigKeys.DFS_WEBHDFS_AUTHENTICATION_FILTER_KEY,
+        DFSConfigKeys.DFS_WEBHDFS_AUTHENTICATION_FILTER_DEFAULT);
+    final String name = className;
+
+    final String pathSpec = WebHdfsFileSystem.PATH_PREFIX + "/*";
+    Map<String, String> params = getAuthFilterParams(conf);
+    HttpServer2.defineFilter(httpServer.getWebAppContext(), name, className,
+        params, new String[] { pathSpec });
+    HttpServer2.LOG.info("Added filter '" + name + "' (class=" + className
+        + ")");
+
+    // add webhdfs packages
+    httpServer.addJerseyResourcePackage(NamenodeWebHdfsMethods.class
+        .getPackage().getName() + ";" + Param.class.getPackage().getName(),
+        pathSpec);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
index 739e701..12adb05 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
@@ -211,13 +211,6 @@ public class WebHdfsFileSystem extends FileSystem
     return super.getCanonicalUri();
   }
 
-  /** Is WebHDFS enabled in conf? */
-  public static boolean isEnabled(final Configuration conf, final Log log) {
-    final boolean b = conf.getBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY,
-        DFSConfigKeys.DFS_WEBHDFS_ENABLED_DEFAULT);
-    return b;
-  }
-
   TokenSelector<DelegationTokenIdentifier> tokenSelector =
       new AbstractDelegationTokenSelector<DelegationTokenIdentifier>(getTokenKind()){};
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c
index b37ebcc..ab6abda 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/native/libhdfs/native_mini_dfs.c
@@ -128,22 +128,6 @@ struct NativeMiniDfsCluster* nmdCreate(struct NativeMiniDfsConf *conf)
             "nmdCreate: new Configuration");
         goto error;
     }
-    if (conf->webhdfsEnabled) {
-        jthr = newJavaStr(env, DFS_WEBHDFS_ENABLED_KEY, &jconfStr);
-        if (jthr) {
-            printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
-                                  "nmdCreate: new String");
-            goto error;
-        }
-        jthr = invokeMethod(env, NULL, INSTANCE, cobj, HADOOP_CONF,
-                            "setBoolean", "(Ljava/lang/String;Z)V",
-                            jconfStr, conf->webhdfsEnabled);
-        if (jthr) {
-            printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
-                                  "nmdCreate: Configuration::setBoolean");
-            goto error;
-        }
-    }
     if (jthr) {
         printExceptionAndFree(env, jthr, PRINT_EXC_ALL,
                               "nmdCreate: Configuration::setBoolean");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java
index 0c3abec..2ff7050 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java
@@ -85,7 +85,6 @@ abstract public class TestSymlinkHdfs extends SymlinkBaseTest {
   @BeforeClass
   public static void beforeClassSetup() throws Exception {
     Configuration conf = new HdfsConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     conf.set(FsPermission.UMASK_LABEL, "000");
     conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_COMPONENT_LENGTH_KEY, 0);
     cluster = new MiniDFSCluster.Builder(conf).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
index da81d2f..5be492f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java
@@ -503,7 +503,6 @@ public class TestDistributedFileSystem {
     RAN.setSeed(seed);
 
     final Configuration conf = getTestConfiguration();
-    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
 
     final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
     final FileSystem hdfs = cluster.getFileSystem();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
index d108d59..163378c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestQuota.java
@@ -851,8 +851,7 @@ public class TestQuota {
     Configuration conf = new HdfsConfiguration();
     final int BLOCK_SIZE = 6 * 1024;
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
-    MiniDFSCluster cluster = 
+    MiniDFSCluster cluster =
       new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     cluster.waitActive();
     FileSystem fs = cluster.getFileSystem();
@@ -913,7 +912,6 @@ public class TestQuota {
     Configuration conf = new HdfsConfiguration();
     final int BLOCK_SIZE = 6 * 1024;
     conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     // Make it relinquish locks. When run serially, the result should
     // be identical.
     conf.setInt(DFSConfigKeys.DFS_CONTENT_SUMMARY_LIMIT_KEY, 2);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java
index 79b02e1..dff8fa4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java
@@ -67,7 +67,6 @@ public class TestDelegationToken {
   @Before
   public void setUp() throws Exception {
     config = new HdfsConfiguration();
-    config.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     config.setLong(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, 10000);
     config.setLong(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, 5000);
     config.setBoolean(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
index e6493a2..3bbd6f1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
@@ -97,7 +97,6 @@ public class TestDelegationTokenForProxyUser {
   @BeforeClass
   public static void setUp() throws Exception {
     config = new HdfsConfiguration();
-    config.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     config.setLong(
         DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, 10000);
     config.setLong(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
index 98297ca..7d06241 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAuditLogs.java
@@ -114,7 +114,6 @@ public class TestAuditLogs {
     final long precision = 1L;
     conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision);
     conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
-    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY, useAsyncLog);
     util = new DFSTestUtil.Builder().setName("TestAuditAllowed").
         setNumFiles(20).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java
index 571d719..55926cc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRespectsBindHostKeys.java
@@ -193,7 +193,6 @@ public class TestNameNodeRespectsBindHostKeys {
 
   private static void setupSsl() throws Exception {
     Configuration conf = new Configuration();
-    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     conf.set(DFSConfigKeys.DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
     conf.set(DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java
index 80369fd..20b25f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestFSMainOperationsWebHdfs.java
@@ -71,7 +71,6 @@ public class TestFSMainOperationsWebHdfs extends FSMainOperationsBaseTest {
   @BeforeClass
   public static void setupCluster() {
     final Configuration conf = new Configuration();
-    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, 1024);
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java
index 7612de3..3405c68 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHttpsFileSystem.java
@@ -52,7 +52,6 @@ public class TestHttpsFileSystem {
   @BeforeClass
   public static void setUp() throws Exception {
     conf = new Configuration();
-    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     conf.set(DFSConfigKeys.DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
     conf.set(DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
     conf.set(DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
index b308607..2d8892c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java
@@ -332,18 +332,6 @@ public class TestWebHDFS {
       }
     }
   }
-  
-  /**
-   * WebHdfs should be enabled by default after HDFS-5532
-   * 
-   * @throws Exception
-   */
-  @Test
-  public void testWebHdfsEnabledByDefault() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    Assert.assertTrue(conf.getBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY,
-        false));
-  }
 
   /**
    * Test snapshot creation through WebHdfs

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
index 027fda0..b2250fe 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsFileSystemContract.java
@@ -60,7 +60,6 @@ public class TestWebHdfsFileSystemContract extends FileSystemContractBaseTest {
   private UserGroupInformation ugi;
 
   static {
-    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     try {
       cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
       cluster.waitActive();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
index d55f2b1..db08325 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsTokens.java
@@ -205,7 +205,6 @@ public class TestWebHdfsTokens {
       String keystoresDir;
       String sslConfDir;
 	    
-      clusterConf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
       clusterConf.set(DFSConfigKeys.DFS_HTTP_POLICY_KEY, HttpConfig.Policy.HTTPS_ONLY.name());
       clusterConf.set(DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
       clusterConf.set(DFSConfigKeys.DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsWithMultipleNameNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsWithMultipleNameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsWithMultipleNameNodes.java
index 11abd2c..aeda32c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsWithMultipleNameNodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHdfsWithMultipleNameNodes.java
@@ -71,8 +71,6 @@ public class TestWebHdfsWithMultipleNameNodes {
       throws Exception {
     LOG.info("nNameNodes=" + nNameNodes + ", nDataNodes=" + nDataNodes);
 
-    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
-
     cluster = new MiniDFSCluster.Builder(conf)
         .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(nNameNodes))
         .numDataNodes(nDataNodes)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/80278a5f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/WebHdfsTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/WebHdfsTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/WebHdfsTestUtil.java
index 369285d..70f9735 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/WebHdfsTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/WebHdfsTestUtil.java
@@ -42,7 +42,6 @@ public class WebHdfsTestUtil {
 
   public static Configuration createConf() {
     final Configuration conf = new Configuration();
-    conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     return conf;
   }
 


[36/51] [abbrv] hadoop git commit: HDFS-7976. Update NFS user guide for mount option 'sync' to minimize or avoid reordered writes. Contributed by Brandon Li

Posted by ka...@apache.org.
HDFS-7976. Update NFS user guide for mount option 'sync' to minimize or avoid reordered writes. Contributed by Brandon Li


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

Branch: refs/heads/YARN-2139
Commit: 51f1f4937402c63392854443ccde18c1579e3348
Parents: 6413d34
Author: Brandon Li <br...@apache.org>
Authored: Tue Mar 24 10:28:38 2015 -0700
Committer: Brandon Li <br...@apache.org>
Committed: Tue Mar 24 10:28:38 2015 -0700

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/51f1f493/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 70be18a..4f3937a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -784,6 +784,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7956. Improve logging for DatanodeRegistration.
     (Plamen Jeliazkov via shv)
 
+    HDFS-7976. Update NFS user guide for mount option "sync" to minimize or
+    avoid reordered writes. (brandonli)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/51f1f493/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 b7e1733..9c95287 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HdfsNfsGateway.md
@@ -126,7 +126,8 @@ It's strongly recommended for the users to update a few configuration properties
         </property>
 
 *   Users are expected to update the file dump directory. NFS client often
-    reorders writes. Sequential writes can arrive at the NFS gateway at random
+    reorders writes, especially when the export is not mounted with "sync" option.
+    Sequential writes can arrive at the NFS gateway at random
     order. This directory is used to temporarily save out-of-order writes
     before writing to HDFS. For each file, the out-of-order writes are dumped after
     they are accumulated to exceed certain threshold (e.g., 1MB) in memory.
@@ -254,13 +255,16 @@ Verify validity of NFS related services
 Mount the export "/"
 --------------------
 
-Currently NFS v3 only uses TCP as the transportation protocol. NLM is not supported so mount option "nolock" is needed. It's recommended to use hard mount. This is because, even after the client sends all data to NFS gateway, it may take NFS gateway some extra time to transfer data to HDFS when writes were reorderd by NFS client Kernel.
+Currently NFS v3 only uses TCP as the transportation protocol. NLM is not supported so mount option "nolock" is needed. 
+Mount option "sync" is strongly recommended since it can minimize or avoid reordered writes, which results in more predictable throughput.
+ Not specifying the sync option may cause unreliable behavior when uploading large files.
+ It's recommended to use hard mount. This is because, even after the client sends all data to NFS gateway, it may take NFS gateway some extra time to transfer data to HDFS when writes were reorderd by NFS client Kernel.
 
 If soft mount has to be used, the user should give it a relatively long timeout (at least no less than the default timeout on the host) .
 
 The users can mount the HDFS namespace as shown below:
 
-     [root]>mount -t nfs -o vers=3,proto=tcp,nolock,noacl $server:/  $mount_point
+     [root]>mount -t nfs -o vers=3,proto=tcp,nolock,noacl,sync $server:/  $mount_point
 
 Then the users can access HDFS as part of the local file system except that, hard link and random write are not supported yet. To optimize the performance of large file I/O, one can increase the NFS transfer size(rsize and wsize) during mount. By default, NFS gateway supports 1MB as the maximum transfer size. For larger data transfer size, one needs to update "nfs.rtmax" and "nfs.rtmax" in hdfs-site.xml.