You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zj...@apache.org on 2015/01/16 19:31:14 UTC

[01/25] hadoop git commit: YARN-2637. Fixed max-am-resource-percent calculation in CapacityScheduler when activating applications. Contributed by Craig Welch

Repository: hadoop
Updated Branches:
  refs/heads/YARN-2928 fe7d0e7c8 -> 630f1cd44


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.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/fifo/TestFifoScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
index b4c4c10..3918bf7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/TestFifoScheduler.java
@@ -143,13 +143,14 @@ public class TestFifoScheduler {
   @Test(timeout=5000)
   public void testAppAttemptMetrics() throws Exception {
     AsyncDispatcher dispatcher = new InlineDispatcher();
+    
+    FifoScheduler scheduler = new FifoScheduler();
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
     RMContext rmContext = new RMContextImpl(dispatcher, null,
-        null, null, null, null, null, null, null, writer);
+        null, null, null, null, null, null, null, writer, scheduler);
     ((RMContextImpl) rmContext).setSystemMetricsPublisher(
         mock(SystemMetricsPublisher.class));
 
-    FifoScheduler scheduler = new FifoScheduler();
     Configuration conf = new Configuration();
     scheduler.setRMContext(rmContext);
     scheduler.init(conf);
@@ -189,12 +190,14 @@ public class TestFifoScheduler {
         new NMTokenSecretManagerInRM(conf);
     nmTokenSecretManager.rollMasterKey();
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
+    
+    FifoScheduler scheduler = new FifoScheduler();
     RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
-        null, containerTokenSecretManager, nmTokenSecretManager, null, writer);
+        null, containerTokenSecretManager, nmTokenSecretManager, null, writer,
+        scheduler);
     ((RMContextImpl) rmContext).setSystemMetricsPublisher(
         mock(SystemMetricsPublisher.class));
 
-    FifoScheduler scheduler = new FifoScheduler();
     scheduler.setRMContext(rmContext);
     scheduler.init(conf);
     scheduler.start();
@@ -260,17 +263,19 @@ public class TestFifoScheduler {
         new NMTokenSecretManagerInRM(conf);
     nmTokenSecretManager.rollMasterKey();
     RMApplicationHistoryWriter writer = mock(RMApplicationHistoryWriter.class);
-    RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
-        null, containerTokenSecretManager, nmTokenSecretManager, null, writer);
-    ((RMContextImpl) rmContext).setSystemMetricsPublisher(
-        mock(SystemMetricsPublisher.class));
-
+    
     FifoScheduler scheduler = new FifoScheduler(){
       @SuppressWarnings("unused")
       public Map<NodeId, FiCaSchedulerNode> getNodes(){
         return nodes;
       }
     };
+    RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
+        null, containerTokenSecretManager, nmTokenSecretManager, null, writer,
+        scheduler);
+    ((RMContextImpl) rmContext).setSystemMetricsPublisher(
+        mock(SystemMetricsPublisher.class));
+
     scheduler.setRMContext(rmContext);
     scheduler.init(conf);
     scheduler.start();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.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/webapp/TestRMWebServicesCapacitySched.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
index c7c403d..ef7435a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesCapacitySched.java
@@ -82,8 +82,6 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
     int numContainers;
     int maxApplications;
     int maxApplicationsPerUser;
-    int maxActiveApplications;
-    int maxActiveApplicationsPerUser;
     int userLimit;
     float userLimitFactor;
   }
@@ -303,10 +301,6 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
           WebServicesTestUtils.getXmlInt(qElem, "maxApplications");
       lqi.maxApplicationsPerUser =
           WebServicesTestUtils.getXmlInt(qElem, "maxApplicationsPerUser");
-      lqi.maxActiveApplications =
-          WebServicesTestUtils.getXmlInt(qElem, "maxActiveApplications");
-      lqi.maxActiveApplicationsPerUser =
-          WebServicesTestUtils.getXmlInt(qElem, "maxActiveApplicationsPerUser");
       lqi.userLimit = WebServicesTestUtils.getXmlInt(qElem, "userLimit");
       lqi.userLimitFactor =
           WebServicesTestUtils.getXmlFloat(qElem, "userLimitFactor");
@@ -386,8 +380,6 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
       lqi.numContainers = info.getInt("numContainers");
       lqi.maxApplications = info.getInt("maxApplications");
       lqi.maxApplicationsPerUser = info.getInt("maxApplicationsPerUser");
-      lqi.maxActiveApplications = info.getInt("maxActiveApplications");
-      lqi.maxActiveApplicationsPerUser = info.getInt("maxActiveApplicationsPerUser");
       lqi.userLimit = info.getInt("userLimit");
       lqi.userLimitFactor = (float) info.getDouble("userLimitFactor");
       verifyLeafQueueGeneric(q, lqi);
@@ -449,10 +441,6 @@ public class TestRMWebServicesCapacitySched extends JerseyTestBase {
         (float)expectedMaxAppsPerUser,
         (float)info.maxApplicationsPerUser, info.userLimitFactor);
 
-    assertTrue("maxActiveApplications doesn't match",
-        info.maxActiveApplications > 0);
-    assertTrue("maxActiveApplicationsPerUser doesn't match",
-        info.maxActiveApplicationsPerUser > 0);
     assertEquals("userLimit doesn't match", csConf.getUserLimit(q),
         info.userLimit);
     assertEquals("userLimitFactor doesn't match",


[02/25] hadoop git commit: YARN-2637. Fixed max-am-resource-percent calculation in CapacityScheduler when activating applications. Contributed by Craig Welch

Posted by zj...@apache.org.
YARN-2637. Fixed max-am-resource-percent calculation in CapacityScheduler when activating applications. Contributed by Craig Welch


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

Branch: refs/heads/YARN-2928
Commit: c53420f58364b11fbda1dace7679d45534533382
Parents: 85aec75
Author: Jian He <ji...@apache.org>
Authored: Tue Jan 13 17:32:07 2015 -0800
Committer: Jian He <ji...@apache.org>
Committed: Tue Jan 13 17:32:07 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../dev-support/findbugs-exclude.xml            |   1 +
 .../resourcemanager/RMActiveServiceContext.java |   4 +-
 .../server/resourcemanager/RMContextImpl.java   |  32 ++-
 .../server/resourcemanager/rmapp/RMApp.java     |   3 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |   5 +
 .../scheduler/capacity/CSQueueUtils.java        |  24 --
 .../scheduler/capacity/LeafQueue.java           | 252 ++++++++++++------
 .../scheduler/common/fica/FiCaSchedulerApp.java |  15 ++
 .../webapp/CapacitySchedulerPage.java           |   4 +-
 .../dao/CapacitySchedulerLeafQueueInfo.java     |  24 +-
 .../applicationsmanager/MockAsm.java            |   7 +
 .../TestAMRMRPCNodeUpdates.java                 |   9 +
 .../TestCapacitySchedulerPlanFollower.java      |   1 +
 .../server/resourcemanager/rmapp/MockRMApp.java |   7 +
 .../capacity/TestApplicationLimits.java         | 256 +++++++++++++------
 .../scheduler/capacity/TestLeafQueue.java       |  79 +++---
 .../scheduler/capacity/TestReservations.java    |  34 +--
 .../scheduler/fifo/TestFifoScheduler.java       |  23 +-
 .../webapp/TestRMWebServicesCapacitySched.java  |  12 -
 20 files changed, 519 insertions(+), 276 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 0afc4cf..bbdfcb5 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -356,6 +356,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3027. Scheduler should use totalAvailable resource from node instead of
     availableResource for maxAllocation. (adhoot via rkanter)
 
+    YARN-2637. Fixed max-am-resource-percent calculation in CapacityScheduler
+    when activating applications. (Craig Welch via jianhe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index 315044d..e0bbd7b 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -173,6 +173,7 @@
       <Field name="userLimit" />
       <Field name="userLimitFactor" />
       <Field name="maxAMResourcePerQueuePercent" />
+      <Field name="lastClusterResource" />
     </Or>
     <Bug pattern="IS2_INCONSISTENT_SYNC" />
   </Match>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.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/RMActiveServiceContext.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
index 3bc2e9b..03fc40e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMActiveServiceContext.java
@@ -117,7 +117,8 @@ public class RMActiveServiceContext {
       RMContainerTokenSecretManager containerTokenSecretManager,
       NMTokenSecretManagerInRM nmTokenSecretManager,
       ClientToAMTokenSecretManagerInRM clientToAMTokenSecretManager,
-      RMApplicationHistoryWriter rmApplicationHistoryWriter) {
+      RMApplicationHistoryWriter rmApplicationHistoryWriter,
+      ResourceScheduler scheduler) {
     this();
     this.setContainerAllocationExpirer(containerAllocationExpirer);
     this.setAMLivelinessMonitor(amLivelinessMonitor);
@@ -128,6 +129,7 @@ public class RMActiveServiceContext {
     this.setNMTokenSecretManager(nmTokenSecretManager);
     this.setClientToAMTokenSecretManager(clientToAMTokenSecretManager);
     this.setRMApplicationHistoryWriter(rmApplicationHistoryWriter);
+    this.setScheduler(scheduler);
 
     RMStateStore nullStore = new NullRMStateStore();
     nullStore.setRMDispatcher(rmDispatcher);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.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/RMContextImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
index ebf2fe4..1d0d6c0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
@@ -87,18 +87,46 @@ public class RMContextImpl implements RMContext {
       RMContainerTokenSecretManager containerTokenSecretManager,
       NMTokenSecretManagerInRM nmTokenSecretManager,
       ClientToAMTokenSecretManagerInRM clientToAMTokenSecretManager,
-      RMApplicationHistoryWriter rmApplicationHistoryWriter) {
+      RMApplicationHistoryWriter rmApplicationHistoryWriter,
+      ResourceScheduler scheduler) {
     this();
     this.setDispatcher(rmDispatcher);
     setActiveServiceContext(new RMActiveServiceContext(rmDispatcher,
         containerAllocationExpirer, amLivelinessMonitor, amFinishingMonitor,
         delegationTokenRenewer, appTokenSecretManager,
         containerTokenSecretManager, nmTokenSecretManager,
-        clientToAMTokenSecretManager, rmApplicationHistoryWriter));
+        clientToAMTokenSecretManager, rmApplicationHistoryWriter,
+        scheduler));
 
     ConfigurationProvider provider = new LocalConfigurationProvider();
     setConfigurationProvider(provider);
   }
+  
+  @VisibleForTesting
+  // helper constructor for tests
+  public RMContextImpl(Dispatcher rmDispatcher,
+      ContainerAllocationExpirer containerAllocationExpirer,
+      AMLivelinessMonitor amLivelinessMonitor,
+      AMLivelinessMonitor amFinishingMonitor,
+      DelegationTokenRenewer delegationTokenRenewer,
+      AMRMTokenSecretManager appTokenSecretManager,
+      RMContainerTokenSecretManager containerTokenSecretManager,
+      NMTokenSecretManagerInRM nmTokenSecretManager,
+      ClientToAMTokenSecretManagerInRM clientToAMTokenSecretManager,
+      RMApplicationHistoryWriter rmApplicationHistoryWriter) {
+    this(
+      rmDispatcher,
+      containerAllocationExpirer,
+      amLivelinessMonitor,
+      amFinishingMonitor,
+      delegationTokenRenewer,
+      appTokenSecretManager,
+      containerTokenSecretManager,
+      nmTokenSecretManager,
+      clientToAMTokenSecretManager,
+      rmApplicationHistoryWriter,
+      null);
+  }
 
   @Override
   public Dispatcher getDispatcher() {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.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/rmapp/RMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
index 624aa18..fbcaab9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMApp.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
@@ -239,4 +240,6 @@ public interface RMApp extends EventHandler<RMAppEvent> {
   RMAppMetrics getRMAppMetrics();
 
   ReservationId getReservationId();
+  
+  ResourceRequest getAMResourceRequest();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.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/rmapp/RMAppImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
index 33b62fe..2d1737a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
@@ -1339,6 +1339,11 @@ public class RMAppImpl implements RMApp, Recoverable {
   public ReservationId getReservationId() {
     return submissionContext.getReservationID();
   }
+  
+  @Override
+  public ResourceRequest getAMResourceRequest() {
+    return this.amReq; 
+  }
 
   protected Credentials parseCredentials() throws IOException {
     Credentials credentials = new Credentials();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.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/CSQueueUtils.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/CSQueueUtils.java
index 0a2fa3a..f458057 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/CSQueueUtils.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/CSQueueUtils.java
@@ -109,30 +109,6 @@ class CSQueueUtils {
     }
     return absoluteMaxCapacityByNodeLabels;
   }
-
-  public static int computeMaxActiveApplications(
-      ResourceCalculator calculator,
-      Resource clusterResource, Resource minimumAllocation, 
-      float maxAMResourcePercent, float absoluteMaxCapacity) {
-    return
-        Math.max(
-            (int)Math.ceil(
-                Resources.ratio(
-                    calculator, 
-                    clusterResource, 
-                    minimumAllocation) * 
-                    maxAMResourcePercent * absoluteMaxCapacity
-                ), 
-            1);
-  }
-
-  public static int computeMaxActiveApplicationsPerUser(
-      int maxActiveApplications, int userLimit, float userLimitFactor) {
-    return Math.max(
-        (int)Math.ceil(
-            maxActiveApplications * (userLimit / 100.0f) * userLimitFactor),
-        1);
-  }
   
   @Lock(CSQueue.class)
   public static void updateQueueStatistics(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/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 dd71069..fd8a7ee 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
@@ -87,9 +87,6 @@ public class LeafQueue extends AbstractCSQueue {
   protected int maxApplicationsPerUser;
   
   private float maxAMResourcePerQueuePercent;
-  private int maxActiveApplications; // Based on absolute max capacity
-  private int maxActiveAppsUsingAbsCap; // Based on absolute capacity
-  private int maxActiveApplicationsPerUser;
   
   private int nodeLocalityDelay;
 
@@ -113,8 +110,15 @@ public class LeafQueue extends AbstractCSQueue {
   // cache last cluster resource to compute actual capacity
   private Resource lastClusterResource = Resources.none();
   
+  // absolute capacity as a resource (based on cluster resource)
+  private Resource absoluteCapacityResource = Resources.none();
+  
   private final QueueHeadroomInfo queueHeadroomInfo = new QueueHeadroomInfo();
   
+  // sum of resources used by application masters for applications
+  // running in this queue
+  private final Resource usedAMResources = Resource.newInstance(0, 0);
+  
   public LeafQueue(CapacitySchedulerContext cs, 
       String queueName, CSQueue parent, CSQueue old) throws IOException {
     super(cs, queueName, parent, old);
@@ -149,19 +153,6 @@ public class LeafQueue extends AbstractCSQueue {
 
     float maxAMResourcePerQueuePercent = cs.getConfiguration()
         .getMaximumApplicationMasterResourcePerQueuePercent(getQueuePath());
-    int maxActiveApplications = 
-        CSQueueUtils.computeMaxActiveApplications(
-            resourceCalculator,
-            cs.getClusterResource(), this.minimumAllocation,
-            maxAMResourcePerQueuePercent, absoluteMaxCapacity);
-    this.maxActiveAppsUsingAbsCap = 
-            CSQueueUtils.computeMaxActiveApplications(
-                resourceCalculator,
-                cs.getClusterResource(), this.minimumAllocation,
-                maxAMResourcePerQueuePercent, absoluteCapacity);
-    int maxActiveApplicationsPerUser =
-        CSQueueUtils.computeMaxActiveApplicationsPerUser(
-            maxActiveAppsUsingAbsCap, userLimit, userLimitFactor);
 
     QueueState state = cs.getConfiguration().getState(getQueuePath());
 
@@ -171,8 +162,7 @@ public class LeafQueue extends AbstractCSQueue {
     setupQueueConfigs(cs.getClusterResource(), capacity, absoluteCapacity,
         maximumCapacity, absoluteMaxCapacity, userLimit, userLimitFactor,
         maxApplications, maxAMResourcePerQueuePercent, maxApplicationsPerUser,
-        maxActiveApplications, maxActiveApplicationsPerUser, state, acls, cs
-            .getConfiguration().getNodeLocalityDelay(), accessibleLabels,
+        state, acls, cs.getConfiguration().getNodeLocalityDelay(), accessibleLabels,
         defaultLabelExpression, this.capacitiyByNodeLabels,
         this.maxCapacityByNodeLabels,
         cs.getConfiguration().getReservationContinueLook());
@@ -200,8 +190,7 @@ public class LeafQueue extends AbstractCSQueue {
       float maximumCapacity, float absoluteMaxCapacity,
       int userLimit, float userLimitFactor,
       int maxApplications, float maxAMResourcePerQueuePercent,
-      int maxApplicationsPerUser, int maxActiveApplications,
-      int maxActiveApplicationsPerUser, QueueState state,
+      int maxApplicationsPerUser, QueueState state,
       Map<QueueACL, AccessControlList> acls, int nodeLocalityDelay,
       Set<String> labels, String defaultLabelExpression,
       Map<String, Float> capacitieByLabel,
@@ -216,6 +205,16 @@ public class LeafQueue extends AbstractCSQueue {
     float absCapacity = getParent().getAbsoluteCapacity() * capacity;
     CSQueueUtils.checkAbsoluteCapacity(getQueueName(), absCapacity,
         absoluteMaxCapacity);
+    
+    this.lastClusterResource = clusterResource;
+    updateAbsoluteCapacityResource(clusterResource);
+    
+    // Initialize headroom info, also used for calculating application 
+    // master resource limits.  Since this happens during queue initialization
+    // and all queues may not be realized yet, we'll use (optimistic) 
+    // absoluteMaxCapacity (it will be replaced with the more accurate 
+    // absoluteMaxAvailCapacity during headroom/userlimit/allocation events)
+    updateHeadroomInfo(clusterResource, absoluteMaxCapacity);
 
     this.absoluteCapacity = absCapacity;
 
@@ -226,9 +225,6 @@ public class LeafQueue extends AbstractCSQueue {
     this.maxAMResourcePerQueuePercent = maxAMResourcePerQueuePercent;
     this.maxApplicationsPerUser = maxApplicationsPerUser;
 
-    this.maxActiveApplications = maxActiveApplications;
-    this.maxActiveApplicationsPerUser = maxActiveApplicationsPerUser;
-
     if (!SchedulerUtils.checkQueueLabelExpression(this.accessibleLabels,
         this.defaultLabelExpression)) {
       throw new IOException("Invalid default label expression of "
@@ -280,21 +276,6 @@ public class LeafQueue extends AbstractCSQueue {
         "maxApplicationsPerUser = " + maxApplicationsPerUser +
         " [= (int)(maxApplications * (userLimit / 100.0f) * " +
         "userLimitFactor) ]" + "\n" +
-        "maxActiveApplications = " + maxActiveApplications +
-        " [= max(" + 
-        "(int)ceil((clusterResourceMemory / minimumAllocation) * " + 
-        "maxAMResourcePerQueuePercent * absoluteMaxCapacity)," + 
-        "1) ]" + "\n" +
-        "maxActiveAppsUsingAbsCap = " + maxActiveAppsUsingAbsCap +
-        " [= max(" + 
-        "(int)ceil((clusterResourceMemory / minimumAllocation) *" + 
-        "maxAMResourcePercent * absoluteCapacity)," + 
-        "1) ]" + "\n" +
-        "maxActiveApplicationsPerUser = " + maxActiveApplicationsPerUser +
-        " [= max(" +
-        "(int)(maxActiveApplications * (userLimit / 100.0f) * " +
-        "userLimitFactor)," +
-        "1) ]" + "\n" +
         "usedCapacity = " + usedCapacity +
         " [= usedResourcesMemory / " +
         "(clusterResourceMemory * absoluteCapacity)]" + "\n" +
@@ -347,14 +328,6 @@ public class LeafQueue extends AbstractCSQueue {
     return maxApplicationsPerUser;
   }
 
-  public synchronized int getMaximumActiveApplications() {
-    return maxActiveApplications;
-  }
-
-  public synchronized int getMaximumActiveApplicationsPerUser() {
-    return maxActiveApplicationsPerUser;
-  }
-
   @Override
   public ActiveUsersManager getActiveUsersManager() {
     return activeUsersManager;
@@ -517,8 +490,6 @@ public class LeafQueue extends AbstractCSQueue {
         newlyParsedLeafQueue.maxApplications,
         newlyParsedLeafQueue.maxAMResourcePerQueuePercent,
         newlyParsedLeafQueue.getMaxApplicationsPerUser(),
-        newlyParsedLeafQueue.getMaximumActiveApplications(), 
-        newlyParsedLeafQueue.getMaximumActiveApplicationsPerUser(),
         newlyParsedLeafQueue.state, newlyParsedLeafQueue.acls,
         newlyParsedLeafQueue.getNodeLocalityDelay(),
         newlyParsedLeafQueue.accessibleLabels,
@@ -604,27 +575,115 @@ public class LeafQueue extends AbstractCSQueue {
     }
 
   }
+  
+  public synchronized Resource getAMResourceLimit() {
+     /* 
+      * The limit to the amount of resources which can be consumed by
+      * application masters for applications running in the queue
+      * is calculated by taking the greater of the max resources currently
+      * available to the queue (see absoluteMaxAvailCapacity) and the absolute
+      * resources guaranteed for the queue and multiplying it by the am
+      * resource percent.
+      *
+      * This is to allow a queue to grow its (proportional) application 
+      * master resource use up to its max capacity when other queues are 
+      * idle but to scale back down to it's guaranteed capacity as they 
+      * become busy.
+      *
+      */
+     Resource queueMaxCap;
+     synchronized (queueHeadroomInfo) {
+       queueMaxCap = queueHeadroomInfo.getQueueMaxCap();
+     }
+     Resource queueCap = Resources.max(resourceCalculator, lastClusterResource,
+       absoluteCapacityResource, queueMaxCap);
+     return Resources.multiplyAndNormalizeUp( 
+          resourceCalculator,
+          queueCap, 
+          maxAMResourcePerQueuePercent, minimumAllocation);
+  }
+  
+  public synchronized Resource getUserAMResourceLimit() {
+     /*
+      * The user amresource limit is based on the same approach as the 
+      * user limit (as it should represent a subset of that).  This means that
+      * it uses the absolute queue capacity instead of the max and is modified
+      * by the userlimit and the userlimit factor as is the userlimit
+      *
+      */ 
+     float effectiveUserLimit = Math.max(userLimit / 100.0f, 1.0f /    
+       Math.max(getActiveUsersManager().getNumActiveUsers(), 1));
+     
+     return Resources.multiplyAndNormalizeUp( 
+          resourceCalculator,
+          absoluteCapacityResource, 
+          maxAMResourcePerQueuePercent * effectiveUserLimit  *
+            userLimitFactor, minimumAllocation);
+  }
 
   private synchronized void activateApplications() {
+    //limit of allowed resource usage for application masters
+    Resource amLimit = getAMResourceLimit();
+    Resource userAMLimit = getUserAMResourceLimit();
+        
     for (Iterator<FiCaSchedulerApp> i=pendingApplications.iterator(); 
          i.hasNext(); ) {
       FiCaSchedulerApp application = i.next();
       
-      // Check queue limit
-      if (getNumActiveApplications() >= getMaximumActiveApplications()) {
-        break;
+      // Check am resource limit
+      Resource amIfStarted = 
+        Resources.add(application.getAMResource(), usedAMResources);
+      
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("application AMResource " + application.getAMResource() +
+          " maxAMResourcePerQueuePercent " + maxAMResourcePerQueuePercent +
+          " amLimit " + amLimit +
+          " lastClusterResource " + lastClusterResource +
+          " amIfStarted " + amIfStarted);
+      }
+      
+      if (!Resources.lessThanOrEqual(
+        resourceCalculator, lastClusterResource, amIfStarted, amLimit)) {
+        if (getNumActiveApplications() < 1) {
+          LOG.warn("maximum-am-resource-percent is insufficient to start a" +
+            " single application in queue, it is likely set too low." +
+            " skipping enforcement to allow at least one application to start"); 
+        } else {
+          LOG.info("not starting application as amIfStarted exceeds amLimit");
+          continue;
+        }
       }
       
-      // Check user limit
+      // Check user am resource limit
+      
       User user = getUser(application.getUser());
-      if (user.getActiveApplications() < getMaximumActiveApplicationsPerUser()) {
-        user.activateApplication();
-        activeApplications.add(application);
-        i.remove();
-        LOG.info("Application " + application.getApplicationId() +
-            " from user: " + application.getUser() + 
-            " activated in queue: " + getQueueName());
+      
+      Resource userAmIfStarted = 
+        Resources.add(application.getAMResource(),
+          user.getConsumedAMResources());
+        
+      if (!Resources.lessThanOrEqual(
+          resourceCalculator, lastClusterResource, userAmIfStarted, 
+          userAMLimit)) {
+        if (getNumActiveApplications() < 1) {
+          LOG.warn("maximum-am-resource-percent is insufficient to start a" +
+            " single application in queue for user, it is likely set too low." +
+            " skipping enforcement to allow at least one application to start"); 
+        } else {
+          LOG.info("not starting application as amIfStarted exceeds " +
+            "userAmLimit");
+          continue;
+        }
       }
+      user.activateApplication();
+      activeApplications.add(application);
+      Resources.addTo(usedAMResources, application.getAMResource());
+      Resources.addTo(user.getConsumedAMResources(), 
+        application.getAMResource());
+      i.remove();
+      LOG.info("Application " + application.getApplicationId() +
+          " from user: " + application.getUser() + 
+          " activated in queue: " + getQueueName());
     }
   }
   
@@ -670,6 +729,10 @@ public class LeafQueue extends AbstractCSQueue {
     boolean wasActive = activeApplications.remove(application);
     if (!wasActive) {
       pendingApplications.remove(application);
+    } else {
+      Resources.subtractFrom(usedAMResources, application.getAMResource());
+      Resources.subtractFrom(user.getConsumedAMResources(),
+        application.getAMResource());
     }
     applicationAttemptMap.remove(application.getApplicationAttemptId());
 
@@ -997,6 +1060,25 @@ public class LeafQueue extends AbstractCSQueue {
     
     return canAssign;
   }
+  
+  private Resource updateHeadroomInfo(Resource clusterResource, 
+      float absoluteMaxAvailCapacity) {
+  
+    Resource queueMaxCap = 
+      Resources.multiplyAndNormalizeDown(
+          resourceCalculator, 
+          clusterResource, 
+          absoluteMaxAvailCapacity,
+          minimumAllocation);
+
+    synchronized (queueHeadroomInfo) {
+      queueHeadroomInfo.setQueueMaxCap(queueMaxCap);
+      queueHeadroomInfo.setClusterResource(clusterResource);
+    }
+    
+    return queueMaxCap;
+    
+  }
 
   @Lock({LeafQueue.class, FiCaSchedulerApp.class})
   Resource computeUserLimitAndSetHeadroom(FiCaSchedulerApp application,
@@ -1015,18 +1097,9 @@ public class LeafQueue extends AbstractCSQueue {
     //capacity
     float absoluteMaxAvailCapacity = CSQueueUtils.getAbsoluteMaxAvailCapacity(
       resourceCalculator, clusterResource, this);
-
-    Resource queueMaxCap =                        // Queue Max-Capacity
-        Resources.multiplyAndNormalizeDown(
-            resourceCalculator, 
-            clusterResource, 
-            absoluteMaxAvailCapacity,
-            minimumAllocation);
-	
-    synchronized (queueHeadroomInfo) {
-      queueHeadroomInfo.setQueueMaxCap(queueMaxCap);
-      queueHeadroomInfo.setClusterResource(clusterResource);
-    }
+    
+    Resource queueMaxCap = 
+      updateHeadroomInfo(clusterResource, absoluteMaxAvailCapacity);
     
     Resource headroom =
         getHeadroom(queueUser, queueMaxCap, clusterResource, userLimit);
@@ -1721,25 +1794,25 @@ public class LeafQueue extends AbstractCSQueue {
         " used=" + usedResources + " numContainers=" + numContainers + 
         " user=" + userName + " user-resources=" + user.getTotalConsumedResources());
   }
+  
+  private void updateAbsoluteCapacityResource(Resource clusterResource) {
+    
+       absoluteCapacityResource = Resources.multiplyAndNormalizeUp( 
+          resourceCalculator,
+          clusterResource, 
+          absoluteCapacity, minimumAllocation);
+       
+  }
 
   @Override
   public synchronized void updateClusterResource(Resource clusterResource) {
     lastClusterResource = clusterResource;
+    updateAbsoluteCapacityResource(clusterResource);
     
-    // Update queue properties
-    maxActiveApplications = 
-        CSQueueUtils.computeMaxActiveApplications(
-            resourceCalculator,
-            clusterResource, minimumAllocation, 
-            maxAMResourcePerQueuePercent, absoluteMaxCapacity);
-    maxActiveAppsUsingAbsCap = 
-        CSQueueUtils.computeMaxActiveApplications(
-            resourceCalculator,
-            clusterResource, minimumAllocation, 
-            maxAMResourcePerQueuePercent, absoluteCapacity);
-    maxActiveApplicationsPerUser = 
-        CSQueueUtils.computeMaxActiveApplicationsPerUser(
-            maxActiveAppsUsingAbsCap, userLimit, userLimitFactor);
+    // Update headroom info based on new cluster resource value
+    // absoluteMaxCapacity now,  will be replaced with absoluteMaxAvailCapacity
+    // during allocation
+    updateHeadroomInfo(clusterResource, absoluteMaxCapacity);
     
     // Update metrics
     CSQueueUtils.updateQueueStatistics(
@@ -1762,6 +1835,7 @@ public class LeafQueue extends AbstractCSQueue {
   @VisibleForTesting
   public static class User {
     Resource consumed = Resources.createResource(0, 0);
+    Resource consumedAMResources = Resources.createResource(0, 0);
     Map<String, Resource> consumedByLabel = new HashMap<String, Resource>();
     int pendingApplications = 0;
     int activeApplications = 0;
@@ -1785,6 +1859,10 @@ public class LeafQueue extends AbstractCSQueue {
     public int getActiveApplications() {
       return activeApplications;
     }
+    
+    public Resource getConsumedAMResources() {
+      return consumedAMResources; 
+    }
 
     public int getTotalApplications() {
       return getPendingApplications() + getActiveApplications();
@@ -1933,6 +2011,10 @@ public class LeafQueue extends AbstractCSQueue {
 
   @Override
   public float getAbsActualCapacity() {
+    //? Is this actually used by anything at present?
+    //  There is a findbugs warning -re lastClusterResource (now excluded),
+    //  when this is used, verify that the access is mt correct and remove
+    //  the findbugs exclusion if possible
     if (Resources.lessThanOrEqual(resourceCalculator, lastClusterResource,
         lastClusterResource, Resources.none())) {
       return absoluteCapacity;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/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 2f9569c..9f97b13 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
@@ -52,6 +52,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerApplicat
 import org.apache.hadoop.yarn.util.resource.Resources;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityHeadroomProvider;
+import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 
 /**
  * Represents an application attempt from the viewpoint of the FIFO or Capacity
@@ -72,6 +73,20 @@ public class FiCaSchedulerApp extends SchedulerApplicationAttempt {
       String user, Queue queue, ActiveUsersManager activeUsersManager,
       RMContext rmContext) {
     super(applicationAttemptId, user, queue, activeUsersManager, rmContext);
+    
+    RMApp rmApp = rmContext.getRMApps().get(getApplicationId());
+    
+    Resource amResource;
+    if (rmApp == null || rmApp.getAMResourceRequest() == null) {
+      //the rmApp may be undefined (the resource manager checks for this too)
+      //and unmanaged applications do not provide an amResource request
+      //in these cases, provide a default using the scheduler
+      amResource = rmContext.getScheduler().getMinimumResourceCapability();
+    } else {
+      amResource = rmApp.getAMResourceRequest().getCapability();
+    }
+    
+    setAMResource(amResource);
   }
 
   synchronized public boolean containerCompleted(RMContainer rmContainer,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.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/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
index 89b4a78..1f65b88 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java
@@ -114,8 +114,8 @@ class CapacitySchedulerPage extends RmView {
           _("Num Containers:", Integer.toString(lqinfo.getNumContainers())).
           _("Max Applications:", Integer.toString(lqinfo.getMaxApplications())).
           _("Max Applications Per User:", Integer.toString(lqinfo.getMaxApplicationsPerUser())).
-          _("Max Schedulable Applications:", Integer.toString(lqinfo.getMaxActiveApplications())).
-          _("Max Schedulable Applications Per User:", Integer.toString(lqinfo.getMaxActiveApplicationsPerUser())).
+          _("Max Application Master Resources:", lqinfo.getAMResourceLimit().toString()).
+          _("Max Application Master Resources Per User:", lqinfo.getUserAMResourceLimit().toString()).
           _("Configured Capacity:", percent(lqinfo.getCapacity() / 100)).
           _("Configured Max Capacity:", percent(lqinfo.getMaxCapacity() / 100)).
           _("Configured Minimum User Limit Percent:", Integer.toString(lqinfo.getUserLimit()) + "%").

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.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/dao/CapacitySchedulerLeafQueueInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
index d90e963..bb4c749 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/CapacitySchedulerLeafQueueInfo.java
@@ -32,11 +32,11 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
   protected int numContainers;
   protected int maxApplications;
   protected int maxApplicationsPerUser;
-  protected int maxActiveApplications;
-  protected int maxActiveApplicationsPerUser;
   protected int userLimit;
   protected UsersInfo users; // To add another level in the XML
   protected float userLimitFactor;
+  protected ResourceInfo aMResourceLimit;
+  protected ResourceInfo userAMResourceLimit;
 
   CapacitySchedulerLeafQueueInfo() {
   };
@@ -48,11 +48,11 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
     numContainers = q.getNumContainers();
     maxApplications = q.getMaxApplications();
     maxApplicationsPerUser = q.getMaxApplicationsPerUser();
-    maxActiveApplications = q.getMaximumActiveApplications();
-    maxActiveApplicationsPerUser = q.getMaximumActiveApplicationsPerUser();
     userLimit = q.getUserLimit();
     users = new UsersInfo(q.getUsers());
     userLimitFactor = q.getUserLimitFactor();
+    aMResourceLimit = new ResourceInfo(q.getAMResourceLimit());
+    userAMResourceLimit = new ResourceInfo(q.getUserAMResourceLimit());
   }
 
   public int getNumActiveApplications() {
@@ -75,14 +75,6 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
     return maxApplicationsPerUser;
   }
 
-  public int getMaxActiveApplications() {
-    return maxActiveApplications;
-  }
-
-  public int getMaxActiveApplicationsPerUser() {
-    return maxActiveApplicationsPerUser;
-  }
-
   public int getUserLimit() {
     return userLimit;
   }
@@ -95,4 +87,12 @@ public class CapacitySchedulerLeafQueueInfo extends CapacitySchedulerQueueInfo {
   public float getUserLimitFactor() {
     return userLimitFactor;
   }
+  
+  public ResourceInfo getAMResourceLimit() {
+    return aMResourceLimit;
+  }
+  
+  public ResourceInfo getUserAMResourceLimit() {
+    return userAMResourceLimit; 
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.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/applicationsmanager/MockAsm.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
index 62e3e5c..f8d92aa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
@@ -51,6 +52,7 @@ import com.google.common.collect.Lists;
 public abstract class MockAsm extends MockApps {
 
   public static class ApplicationBase implements RMApp {
+    ResourceRequest amReq;
     @Override
     public String getUser() {
       throw new UnsupportedOperationException("Not supported yet.");
@@ -183,6 +185,11 @@ public abstract class MockAsm extends MockApps {
     public ReservationId getReservationId() {
       throw new UnsupportedOperationException("Not supported yet.");
     }
+    
+    @Override
+    public ResourceRequest getAMResourceRequest() {
+      return this.amReq; 
+    }
   }
 
   public static RMApp newApplication(int i) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCNodeUpdates.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/applicationsmanager/TestAMRMRPCNodeUpdates.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCNodeUpdates.java
index e93d351..f4cb3b3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCNodeUpdates.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRMRPCNodeUpdates.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.junit.Assert;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
@@ -41,6 +42,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -55,6 +57,13 @@ public class TestAMRMRPCNodeUpdates {
     dispatcher = new DrainDispatcher();
     this.rm = new MockRM() {
       @Override
+      public void init(Configuration conf) {
+        conf.set(
+          CapacitySchedulerConfiguration.MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT,
+          "1.0");
+        super.init(conf);
+      }
+      @Override
       protected EventHandler<SchedulerEvent> createSchedulerEventDispatcher() {
         return new SchedulerEventDispatcher(this.scheduler) {
           @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.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/reservation/TestCapacitySchedulerPlanFollower.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java
index c7513ab..b8663f6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacitySchedulerPlanFollower.java
@@ -82,6 +82,7 @@ public class TestCapacitySchedulerPlanFollower extends TestSchedulerPlanFollower
         .thenReturn(null);
     Mockito.doReturn(rmApp).when(spyApps).get((ApplicationId) Matchers.any());
     when(spyRMContext.getRMApps()).thenReturn(spyApps);
+    when(spyRMContext.getScheduler()).thenReturn(scheduler);
 
     CapacitySchedulerConfiguration csConf =
         new CapacitySchedulerConfiguration();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.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/rmapp/MockRMApp.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
index 787b5d7..ec990f9 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/MockRMApp.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.ReservationId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.ResourceRequest;
 import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationSubmissionContextPBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -55,6 +56,7 @@ public class MockRMApp implements RMApp {
   StringBuilder diagnostics = new StringBuilder();
   RMAppAttempt attempt;
   int maxAppAttempts = 1;
+  ResourceRequest amReq;
 
   public MockRMApp(int newid, long time, RMAppState newState) {
     finish = time;
@@ -264,4 +266,9 @@ public class MockRMApp implements RMApp {
   public ReservationId getReservationId() {
     throw new UnsupportedOperationException("Not supported yet.");
   }
+  
+  @Override
+  public ResourceRequest getAMResourceRequest() {
+    return this.amReq; 
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestApplicationLimits.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/TestApplicationLimits.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/TestApplicationLimits.java
index 0cd74d0..81a5aad 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/TestApplicationLimits.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/TestApplicationLimits.java
@@ -28,16 +28,21 @@ import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
 
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -47,8 +52,10 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 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.rmapp.RMApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
+import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ActiveUsersManager;
 import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator;
 import org.apache.hadoop.yarn.util.resource.ResourceCalculator;
@@ -56,6 +63,7 @@ import org.apache.hadoop.yarn.util.resource.Resources;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
+import org.junit.Ignore;
 
 public class TestApplicationLimits {
   
@@ -119,8 +127,6 @@ public class TestApplicationLimits {
     // Some default values
     doReturn(100).when(queue).getMaxApplications();
     doReturn(25).when(queue).getMaxApplicationsPerUser();
-    doReturn(10).when(queue).getMaximumActiveApplications();
-    doReturn(2).when(queue).getMaximumActiveApplicationsPerUser();
   }
   
   private static final String A = "a";
@@ -136,10 +142,14 @@ public class TestApplicationLimits {
     final String Q_B = CapacitySchedulerConfiguration.ROOT + "." + B;
     conf.setCapacity(Q_B, 90);
     
+    conf.setUserLimit(CapacitySchedulerConfiguration.ROOT + "." + A, 50);
+    conf.setUserLimitFactor(CapacitySchedulerConfiguration.ROOT + "." + A, 5.0f);
+    
     LOG.info("Setup top-level queues a and b");
   }
 
-  private FiCaSchedulerApp getMockApplication(int appId, String user) {
+  private FiCaSchedulerApp getMockApplication(int appId, String user,
+    Resource amResource) {
     FiCaSchedulerApp application = mock(FiCaSchedulerApp.class);
     ApplicationAttemptId applicationAttemptId =
         TestUtils.getMockApplicationAttemptId(appId, 0);
@@ -147,10 +157,90 @@ public class TestApplicationLimits {
         when(application).getApplicationId();
     doReturn(applicationAttemptId). when(application).getApplicationAttemptId();
     doReturn(user).when(application).getUser();
+    doReturn(amResource).when(application).getAMResource();
     return application;
   }
   
   @Test
+  public void testAMResourceLimit() throws Exception {
+    final String user_0 = "user_0";
+    final String user_1 = "user_1";
+    
+    // This uses the default 10% of cluster value for the max am resources
+    // which are allowed, at 80GB = 8GB for AM's at the queue level.  The user
+    // am limit is 4G initially (based on the queue absolute capacity)
+    // when there is only 1 user, and drops to 2G (the userlimit) when there
+    // is a second user
+    queue.updateClusterResource(Resource.newInstance(80 * GB, 40));
+    
+    ActiveUsersManager activeUsersManager = mock(ActiveUsersManager.class);
+    when(queue.getActiveUsersManager()).thenReturn(activeUsersManager);
+    
+    assertEquals(Resource.newInstance(8 * GB, 1), queue.getAMResourceLimit());
+    assertEquals(Resource.newInstance(4 * GB, 1),
+      queue.getUserAMResourceLimit());
+    
+    // Two apps for user_0, both start
+    int APPLICATION_ID = 0;
+    FiCaSchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0, 
+      Resource.newInstance(2 * GB, 1));
+    queue.submitApplicationAttempt(app_0, user_0);
+    assertEquals(1, queue.getNumActiveApplications());
+    assertEquals(0, queue.getNumPendingApplications());
+    assertEquals(1, queue.getNumActiveApplications(user_0));
+    assertEquals(0, queue.getNumPendingApplications(user_0));
+    
+    when(activeUsersManager.getNumActiveUsers()).thenReturn(1);
+
+    FiCaSchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0, 
+      Resource.newInstance(2 * GB, 1));
+    queue.submitApplicationAttempt(app_1, user_0);
+    assertEquals(2, queue.getNumActiveApplications());
+    assertEquals(0, queue.getNumPendingApplications());
+    assertEquals(2, queue.getNumActiveApplications(user_0));
+    assertEquals(0, queue.getNumPendingApplications(user_0));
+    
+    // AMLimits unchanged
+    assertEquals(Resource.newInstance(8 * GB, 1), queue.getAMResourceLimit());
+    assertEquals(Resource.newInstance(4 * GB, 1),
+      queue.getUserAMResourceLimit());
+    
+    // One app for user_1, starts
+    FiCaSchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_1, 
+      Resource.newInstance(2 * GB, 1));
+    queue.submitApplicationAttempt(app_2, user_1);
+    assertEquals(3, queue.getNumActiveApplications());
+    assertEquals(0, queue.getNumPendingApplications());
+    assertEquals(1, queue.getNumActiveApplications(user_1));
+    assertEquals(0, queue.getNumPendingApplications(user_1));
+    
+    when(activeUsersManager.getNumActiveUsers()).thenReturn(2);
+    
+    // Now userAMResourceLimit drops to the queue configured 50% as there is
+    // another user active
+    assertEquals(Resource.newInstance(8 * GB, 1), queue.getAMResourceLimit());
+    assertEquals(Resource.newInstance(2 * GB, 1),
+      queue.getUserAMResourceLimit());
+    
+    // Second user_1 app cannot start
+    FiCaSchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_1, 
+      Resource.newInstance(2 * GB, 1));
+    queue.submitApplicationAttempt(app_3, user_1);
+    assertEquals(3, queue.getNumActiveApplications());
+    assertEquals(1, queue.getNumPendingApplications());
+    assertEquals(1, queue.getNumActiveApplications(user_1));
+    assertEquals(1, queue.getNumPendingApplications(user_1));
+
+    // Now finish app so another should be activated
+    queue.finishApplicationAttempt(app_2, A);
+    assertEquals(3, queue.getNumActiveApplications());
+    assertEquals(0, queue.getNumPendingApplications());
+    assertEquals(1, queue.getNumActiveApplications(user_1));
+    assertEquals(0, queue.getNumPendingApplications(user_1));
+    
+  }
+  
+  @Test
   public void testLimitsComputation() throws Exception {
     CapacitySchedulerConfiguration csConf = 
         new CapacitySchedulerConfiguration();
@@ -172,7 +262,8 @@ public class TestApplicationLimits {
     when(csContext.getRMContext()).thenReturn(rmContext);
     
     // Say cluster has 100 nodes of 16G each
-    Resource clusterResource = Resources.createResource(100 * 16 * GB, 100 * 16);
+    Resource clusterResource = 
+      Resources.createResource(100 * 16 * GB, 100 * 16);
     when(csContext.getClusterResource()).thenReturn(clusterResource);
     
     Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
@@ -183,28 +274,14 @@ public class TestApplicationLimits {
     LeafQueue queue = (LeafQueue)queues.get(A);
     
     LOG.info("Queue 'A' -" +
-    		" maxActiveApplications=" + queue.getMaximumActiveApplications() + 
-    		" maxActiveApplicationsPerUser=" + 
-    		queue.getMaximumActiveApplicationsPerUser());
-    int expectedMaxActiveApps = 
-        Math.max(1, 
-            (int)Math.ceil(((float)clusterResource.getMemory() / (1*GB)) * 
-                   csConf.
-                     getMaximumApplicationMasterResourcePerQueuePercent(
-                                                        queue.getQueuePath()) *
-                   queue.getAbsoluteMaximumCapacity()));
-    assertEquals(expectedMaxActiveApps, 
-                 queue.getMaximumActiveApplications());
-    int expectedMaxActiveAppsUsingAbsCap = 
-            Math.max(1, 
-                (int)Math.ceil(((float)clusterResource.getMemory() / (1*GB)) * 
-                       csConf.getMaximumApplicationMasterResourcePercent() *
-                       queue.getAbsoluteCapacity()));
-    assertEquals(
-        (int)Math.ceil(
-        		expectedMaxActiveAppsUsingAbsCap * (queue.getUserLimit() / 100.0f) * 
-            queue.getUserLimitFactor()), 
-        queue.getMaximumActiveApplicationsPerUser());
+    		" aMResourceLimit=" + queue.getAMResourceLimit() + 
+    		" UserAMResourceLimit=" + 
+    		queue.getUserAMResourceLimit());
+    
+    assertEquals(queue.getAMResourceLimit(), Resource.newInstance(160*GB, 1));
+    assertEquals(queue.getUserAMResourceLimit(), 
+      Resource.newInstance(80*GB, 1));
+    
     assertEquals(
         (int)(clusterResource.getMemory() * queue.getAbsoluteCapacity()),
         queue.getMetrics().getAvailableMB()
@@ -213,24 +290,11 @@ public class TestApplicationLimits {
     // Add some nodes to the cluster & test new limits
     clusterResource = Resources.createResource(120 * 16 * GB);
     root.updateClusterResource(clusterResource);
-    expectedMaxActiveApps = 
-        Math.max(1, 
-            (int)Math.ceil(((float)clusterResource.getMemory() / (1*GB)) * 
-                   csConf.
-                     getMaximumApplicationMasterResourcePerQueuePercent(
-                                                        queue.getQueuePath()) *
-                   queue.getAbsoluteMaximumCapacity()));
-    assertEquals(expectedMaxActiveApps, 
-                 queue.getMaximumActiveApplications());
-    expectedMaxActiveAppsUsingAbsCap = 
-            Math.max(1, 
-                (int)Math.ceil(((float)clusterResource.getMemory() / (1*GB)) * 
-                       csConf.getMaximumApplicationMasterResourcePercent() *
-                       queue.getAbsoluteCapacity()));
-    assertEquals(
-        (int)Math.ceil(expectedMaxActiveAppsUsingAbsCap * 
-            (queue.getUserLimit() / 100.0f) * queue.getUserLimitFactor()), 
-        queue.getMaximumActiveApplicationsPerUser());
+    
+    assertEquals(queue.getAMResourceLimit(), Resource.newInstance(192*GB, 1));
+    assertEquals(queue.getUserAMResourceLimit(), 
+      Resource.newInstance(96*GB, 1));
+    
     assertEquals(
         (int)(clusterResource.getMemory() * queue.getAbsoluteCapacity()),
         queue.getMetrics().getAvailableMB()
@@ -271,18 +335,15 @@ public class TestApplicationLimits {
     clusterResource = Resources.createResource(100 * 16 * GB);
 
     queue = (LeafQueue)queues.get(A);
-    expectedMaxActiveApps = 
-        Math.max(1, 
-            (int)Math.ceil(((float)clusterResource.getMemory() / (1*GB)) * 
-                   csConf.
-                     getMaximumApplicationMasterResourcePerQueuePercent(
-                                                        queue.getQueuePath()) *
-                   queue.getAbsoluteMaximumCapacity()));
 
     assertEquals((long) 0.5, 
-        (long) csConf.getMaximumApplicationMasterResourcePerQueuePercent(queue.getQueuePath()));
-    assertEquals(expectedMaxActiveApps, 
-        queue.getMaximumActiveApplications());
+        (long) csConf.getMaximumApplicationMasterResourcePerQueuePercent(
+          queue.getQueuePath())
+        );
+    
+    assertEquals(queue.getAMResourceLimit(), Resource.newInstance(800*GB, 1));
+    assertEquals(queue.getUserAMResourceLimit(), 
+      Resource.newInstance(400*GB, 1));
 
     // Change the per-queue max applications.
     csConf.setInt(
@@ -308,10 +369,16 @@ public class TestApplicationLimits {
   public void testActiveApplicationLimits() throws Exception {
     final String user_0 = "user_0";
     final String user_1 = "user_1";
+    final String user_2 = "user_2";
+    
+    assertEquals(Resource.newInstance(16 * GB, 1), queue.getAMResourceLimit());
+    assertEquals(Resource.newInstance(8 * GB, 1),
+      queue.getUserAMResourceLimit());
     
     int APPLICATION_ID = 0;
     // Submit first application
-    FiCaSchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0);
+    FiCaSchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0,
+      Resources.createResource(4 * GB, 0));
     queue.submitApplicationAttempt(app_0, user_0);
     assertEquals(1, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
@@ -319,15 +386,17 @@ public class TestApplicationLimits {
     assertEquals(0, queue.getNumPendingApplications(user_0));
 
     // Submit second application
-    FiCaSchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0);
+    FiCaSchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0,
+      Resources.createResource(4 * GB, 0));
     queue.submitApplicationAttempt(app_1, user_0);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
     assertEquals(0, queue.getNumPendingApplications(user_0));
     
-    // Submit third application, should remain pending
-    FiCaSchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_0);
+    // Submit third application, should remain pending due to user amlimit
+    FiCaSchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_0,
+      Resources.createResource(4 * GB, 0));
     queue.submitApplicationAttempt(app_2, user_0);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(1, queue.getNumPendingApplications());
@@ -342,18 +411,17 @@ public class TestApplicationLimits {
     assertEquals(0, queue.getNumPendingApplications(user_0));
     
     // Submit another one for user_0
-    FiCaSchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_0);
+    FiCaSchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_0,
+      Resources.createResource(4 * GB, 0));
     queue.submitApplicationAttempt(app_3, user_0);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(1, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
     assertEquals(1, queue.getNumPendingApplications(user_0));
     
-    // Change queue limit to be smaller so 2 users can fill it up
-    doReturn(3).when(queue).getMaximumActiveApplications();
-    
     // Submit first app for user_1
-    FiCaSchedulerApp app_4 = getMockApplication(APPLICATION_ID++, user_1);
+    FiCaSchedulerApp app_4 = getMockApplication(APPLICATION_ID++, user_1,
+      Resources.createResource(8 * GB, 0));
     queue.submitApplicationAttempt(app_4, user_1);
     assertEquals(3, queue.getNumActiveApplications());
     assertEquals(1, queue.getNumPendingApplications());
@@ -362,15 +430,17 @@ public class TestApplicationLimits {
     assertEquals(1, queue.getNumActiveApplications(user_1));
     assertEquals(0, queue.getNumPendingApplications(user_1));
 
-    // Submit second app for user_1, should block due to queue-limit
-    FiCaSchedulerApp app_5 = getMockApplication(APPLICATION_ID++, user_1);
-    queue.submitApplicationAttempt(app_5, user_1);
+    // Submit first app for user_2, should block due to queue amlimit
+    FiCaSchedulerApp app_5 = getMockApplication(APPLICATION_ID++, user_2,
+      Resources.createResource(8 * GB, 0));
+    queue.submitApplicationAttempt(app_5, user_2);
     assertEquals(3, queue.getNumActiveApplications());
     assertEquals(2, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
     assertEquals(1, queue.getNumPendingApplications(user_0));
     assertEquals(1, queue.getNumActiveApplications(user_1));
-    assertEquals(1, queue.getNumPendingApplications(user_1));
+    assertEquals(0, queue.getNumPendingApplications(user_1));
+    assertEquals(1, queue.getNumPendingApplications(user_2));
 
     // Now finish one app of user_1 so app_5 should be activated
     queue.finishApplicationAttempt(app_4, A);
@@ -378,21 +448,22 @@ public class TestApplicationLimits {
     assertEquals(1, queue.getNumPendingApplications());
     assertEquals(2, queue.getNumActiveApplications(user_0));
     assertEquals(1, queue.getNumPendingApplications(user_0));
-    assertEquals(1, queue.getNumActiveApplications(user_1));
+    assertEquals(0, queue.getNumActiveApplications(user_1));
     assertEquals(0, queue.getNumPendingApplications(user_1));
+    assertEquals(1, queue.getNumActiveApplications(user_2));
+    assertEquals(0, queue.getNumPendingApplications(user_2));
+    
   }
-
+  
   @Test
   public void testActiveLimitsWithKilledApps() throws Exception {
     final String user_0 = "user_0";
 
     int APPLICATION_ID = 0;
 
-    // set max active to 2
-    doReturn(2).when(queue).getMaximumActiveApplications();
-
     // Submit first application
-    FiCaSchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0);
+    FiCaSchedulerApp app_0 = getMockApplication(APPLICATION_ID++, user_0,
+      Resources.createResource(4 * GB, 0));
     queue.submitApplicationAttempt(app_0, user_0);
     assertEquals(1, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
@@ -401,7 +472,8 @@ public class TestApplicationLimits {
     assertTrue(queue.activeApplications.contains(app_0));
 
     // Submit second application
-    FiCaSchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0);
+    FiCaSchedulerApp app_1 = getMockApplication(APPLICATION_ID++, user_0,
+      Resources.createResource(4 * GB, 0));
     queue.submitApplicationAttempt(app_1, user_0);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(0, queue.getNumPendingApplications());
@@ -410,7 +482,8 @@ public class TestApplicationLimits {
     assertTrue(queue.activeApplications.contains(app_1));
 
     // Submit third application, should remain pending
-    FiCaSchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_0);
+    FiCaSchedulerApp app_2 = getMockApplication(APPLICATION_ID++, user_0,
+      Resources.createResource(4 * GB, 0));
     queue.submitApplicationAttempt(app_2, user_0);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(1, queue.getNumPendingApplications());
@@ -419,7 +492,8 @@ public class TestApplicationLimits {
     assertTrue(queue.pendingApplications.contains(app_2));
 
     // Submit fourth application, should remain pending
-    FiCaSchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_0);
+    FiCaSchedulerApp app_3 = getMockApplication(APPLICATION_ID++, user_0,
+      Resources.createResource(4 * GB, 0));
     queue.submitApplicationAttempt(app_3, user_0);
     assertEquals(2, queue.getNumActiveApplications());
     assertEquals(2, queue.getNumPendingApplications());
@@ -506,6 +580,18 @@ public class TestApplicationLimits {
     RecordFactory recordFactory = 
         RecordFactoryProvider.getRecordFactory(null);
     RMContext rmContext = TestUtils.getMockRMContext();
+    RMContext spyRMContext = spy(rmContext);
+    
+    ConcurrentMap<ApplicationId, RMApp> spyApps = 
+        spy(new ConcurrentHashMap<ApplicationId, RMApp>());
+    RMApp rmApp = mock(RMApp.class);
+    ResourceRequest amResourceRequest = mock(ResourceRequest.class);
+    Resource amResource = Resources.createResource(0, 0);
+    when(amResourceRequest.getCapability()).thenReturn(amResource);
+    when(rmApp.getAMResourceRequest()).thenReturn(amResourceRequest);
+    Mockito.doReturn(rmApp).when(spyApps).get((ApplicationId)Matchers.any());
+    when(spyRMContext.getRMApps()).thenReturn(spyApps);
+    
 
     Priority priority_1 = TestUtils.createMockPriority(1);
 
@@ -513,9 +599,9 @@ public class TestApplicationLimits {
     // and check headroom
     final ApplicationAttemptId appAttemptId_0_0 = 
         TestUtils.getMockApplicationAttemptId(0, 0); 
-    FiCaSchedulerApp app_0_0 = 
-        spy(new FiCaSchedulerApp(appAttemptId_0_0, user_0, queue, 
-            queue.getActiveUsersManager(), rmContext));
+    FiCaSchedulerApp app_0_0 = new FiCaSchedulerApp(
+      appAttemptId_0_0, user_0, queue, 
+            queue.getActiveUsersManager(), spyRMContext);
     queue.submitApplicationAttempt(app_0_0, user_0);
 
     List<ResourceRequest> app_0_0_requests = new ArrayList<ResourceRequest>();
@@ -532,9 +618,9 @@ public class TestApplicationLimits {
     // Submit second application from user_0, check headroom
     final ApplicationAttemptId appAttemptId_0_1 = 
         TestUtils.getMockApplicationAttemptId(1, 0); 
-    FiCaSchedulerApp app_0_1 = 
-        spy(new FiCaSchedulerApp(appAttemptId_0_1, user_0, queue, 
-            queue.getActiveUsersManager(), rmContext));
+    FiCaSchedulerApp app_0_1 = new FiCaSchedulerApp(
+      appAttemptId_0_1, user_0, queue, 
+            queue.getActiveUsersManager(), spyRMContext);
     queue.submitApplicationAttempt(app_0_1, user_0);
     
     List<ResourceRequest> app_0_1_requests = new ArrayList<ResourceRequest>();
@@ -551,9 +637,9 @@ public class TestApplicationLimits {
     // Submit first application from user_1, check  for new headroom
     final ApplicationAttemptId appAttemptId_1_0 = 
         TestUtils.getMockApplicationAttemptId(2, 0); 
-    FiCaSchedulerApp app_1_0 = 
-        spy(new FiCaSchedulerApp(appAttemptId_1_0, user_1, queue, 
-            queue.getActiveUsersManager(), rmContext));
+    FiCaSchedulerApp app_1_0 = new FiCaSchedulerApp(
+      appAttemptId_1_0, user_1, queue, 
+            queue.getActiveUsersManager(), spyRMContext);
     queue.submitApplicationAttempt(app_1_0, user_1);
 
     List<ResourceRequest> app_1_0_requests = new ArrayList<ResourceRequest>();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestLeafQueue.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/TestLeafQueue.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/TestLeafQueue.java
index fb7bb2c..ead5719 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/TestLeafQueue.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/TestLeafQueue.java
@@ -101,6 +101,7 @@ public class TestLeafQueue {
 
   RMContext rmContext;
   RMContext spyRMContext;
+  ResourceRequest amResourceRequest;
   CapacityScheduler cs;
   CapacitySchedulerConfiguration csConf;
   CapacitySchedulerContext csContext;
@@ -124,6 +125,10 @@ public class TestLeafQueue {
         spy(new ConcurrentHashMap<ApplicationId, RMApp>());
     RMApp rmApp = mock(RMApp.class);
     when(rmApp.getRMAppAttempt((ApplicationAttemptId)Matchers.any())).thenReturn(null);
+    amResourceRequest = mock(ResourceRequest.class);
+    when(amResourceRequest.getCapability()).thenReturn(
+      Resources.createResource(0, 0));
+    when(rmApp.getAMResourceRequest()).thenReturn(amResourceRequest);
     Mockito.doReturn(rmApp).when(spyApps).get((ApplicationId)Matchers.any());
     when(spyRMContext.getRMApps()).thenReturn(spyApps);
     
@@ -265,26 +270,37 @@ public class TestLeafQueue {
   
   @Test
   public void testInitializeQueue() throws Exception {
-	  final float epsilon = 1e-5f;
-	  //can add more sturdy test with 3-layer queues 
-	  //once MAPREDUCE:3410 is resolved
-	  LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A));
-	  assertEquals(0.085, a.getCapacity(), epsilon);
-	  assertEquals(0.085, a.getAbsoluteCapacity(), epsilon);
-	  assertEquals(0.2, a.getMaximumCapacity(), epsilon);
-	  assertEquals(0.2, a.getAbsoluteMaximumCapacity(), epsilon);
+    final float epsilon = 1e-5f;
+    //can add more sturdy test with 3-layer queues 
+    //once MAPREDUCE:3410 is resolved
+    LeafQueue a = stubLeafQueue((LeafQueue)queues.get(A));
+    assertEquals(0.085, a.getCapacity(), epsilon);
+    assertEquals(0.085, a.getAbsoluteCapacity(), epsilon);
+    assertEquals(0.2, a.getMaximumCapacity(), epsilon);
+    assertEquals(0.2, a.getAbsoluteMaximumCapacity(), epsilon);
+    
+    LeafQueue b = stubLeafQueue((LeafQueue)queues.get(B));
+    assertEquals(0.80, b.getCapacity(), epsilon);
+    assertEquals(0.80, b.getAbsoluteCapacity(), epsilon);
+    assertEquals(0.99, b.getMaximumCapacity(), epsilon);
+    assertEquals(0.99, b.getAbsoluteMaximumCapacity(), epsilon);
+    
+    ParentQueue c = (ParentQueue)queues.get(C);
+    assertEquals(0.015, c.getCapacity(), epsilon);
+    assertEquals(0.015, c.getAbsoluteCapacity(), epsilon);
+    assertEquals(0.1, c.getMaximumCapacity(), epsilon);
+    assertEquals(0.1, c.getAbsoluteMaximumCapacity(), epsilon);
+
+	  //Verify the value for getAMResourceLimit for queues with < .1 maxcap
+	  Resource clusterResource = Resource.newInstance(50 * GB, 50);
 	  
-	  LeafQueue b = stubLeafQueue((LeafQueue)queues.get(B));
-	  assertEquals(0.80, b.getCapacity(), epsilon);
-	  assertEquals(0.80, b.getAbsoluteCapacity(), epsilon);
-	  assertEquals(0.99, b.getMaximumCapacity(), epsilon);
-	  assertEquals(0.99, b.getAbsoluteMaximumCapacity(), epsilon);
-
-	  ParentQueue c = (ParentQueue)queues.get(C);
-	  assertEquals(0.015, c.getCapacity(), epsilon);
-	  assertEquals(0.015, c.getAbsoluteCapacity(), epsilon);
-	  assertEquals(0.1, c.getMaximumCapacity(), epsilon);
-	  assertEquals(0.1, c.getAbsoluteMaximumCapacity(), epsilon);
+	  a.updateClusterResource(clusterResource);
+	  assertEquals(Resource.newInstance(1 * GB, 1), 
+	    a.getAMResourceLimit());
+    
+	  b.updateClusterResource(clusterResource);
+	  assertEquals(Resource.newInstance(5 * GB, 1), 
+	    b.getAMResourceLimit());
   }
  
   @Test
@@ -679,7 +695,7 @@ public class TestLeafQueue {
               TestUtils.getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 =
         new FiCaSchedulerApp(appAttemptId_0, user_0, qb,
-            qb.getActiveUsersManager(), rmContext);
+            qb.getActiveUsersManager(), spyRMContext);
     qb.submitApplicationAttempt(app_0, user_0);
     Priority u0Priority = TestUtils.createMockPriority(1);
     app_0.updateResourceRequests(Collections.singletonList(
@@ -702,7 +718,7 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(2, 0);
     FiCaSchedulerApp app_2 =
         new FiCaSchedulerApp(appAttemptId_2, user_1, qb,
-            qb.getActiveUsersManager(), rmContext);
+            qb.getActiveUsersManager(), spyRMContext);
     Priority u1Priority = TestUtils.createMockPriority(2);
     app_2.updateResourceRequests(Collections.singletonList(
         TestUtils.createResourceRequest(ResourceRequest.ANY, 4*GB, 1, true,
@@ -736,12 +752,12 @@ public class TestLeafQueue {
         TestUtils.getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 =
         new FiCaSchedulerApp(appAttemptId_1, user_0, qb,
-            qb.getActiveUsersManager(), rmContext);
+            qb.getActiveUsersManager(), spyRMContext);
     final ApplicationAttemptId appAttemptId_3 =
         TestUtils.getMockApplicationAttemptId(3, 0);
     FiCaSchedulerApp app_3 =
         new FiCaSchedulerApp(appAttemptId_3, user_1, qb,
-            qb.getActiveUsersManager(), rmContext);
+            qb.getActiveUsersManager(), spyRMContext);
     app_1.updateResourceRequests(Collections.singletonList(
         TestUtils.createResourceRequest(ResourceRequest.ANY, 2*GB, 1, true,
             u0Priority, recordFactory)));
@@ -764,7 +780,7 @@ public class TestLeafQueue {
               TestUtils.getMockApplicationAttemptId(4, 0);
     FiCaSchedulerApp app_4 =
               new FiCaSchedulerApp(appAttemptId_4, user_0, qb,
-                      qb.getActiveUsersManager(), rmContext);
+                      qb.getActiveUsersManager(), spyRMContext);
     qb.submitApplicationAttempt(app_4, user_0);
     app_4.updateResourceRequests(Collections.singletonList(
               TestUtils.createResourceRequest(ResourceRequest.ANY, 6*GB, 1, true,
@@ -980,7 +996,6 @@ public class TestLeafQueue {
     assertEquals(0*GB, app_1.getHeadroom().getMemory());
     
     // Check headroom for app_2 
-    LOG.info("here");
     app_1.updateResourceRequests(Collections.singletonList(     // unset
         TestUtils.createResourceRequest(ResourceRequest.ANY, 1*GB, 0, true,
             priority, recordFactory)));
@@ -1904,6 +1919,9 @@ public class TestLeafQueue {
 
     // Users
     final String user_e = "user_e";
+    
+    when(amResourceRequest.getCapability()).thenReturn(
+      Resources.createResource(1 * GB, 0));
 
     // Submit applications
     final ApplicationAttemptId appAttemptId_0 =
@@ -1942,7 +1960,7 @@ public class TestLeafQueue {
             newQueues, queues,
             TestUtils.spyHook);
     queues = newQueues;
-    root.reinitialize(newRoot, cs.getClusterResource());
+    root.reinitialize(newRoot, csContext.getClusterResource());
 
     // after reinitialization
     assertEquals(3, e.activeApplications.size());
@@ -1982,6 +2000,9 @@ public class TestLeafQueue {
 
     // Users
     final String user_e = "user_e";
+    
+    when(amResourceRequest.getCapability()).thenReturn(
+      Resources.createResource(1 * GB, 0));
 
     // Submit applications
     final ApplicationAttemptId appAttemptId_0 =
@@ -2291,20 +2312,20 @@ public class TestLeafQueue {
     csConf.setCapacity(CapacitySchedulerConfiguration.ROOT + "." + A, 80);
     LeafQueue a = new LeafQueue(csContext, A, root, null);
     assertEquals(0.1f, a.getMaxAMResourcePerQueuePercent(), 1e-3f);
-    assertEquals(160, a.getMaximumActiveApplications());
+    assertEquals(a.getAMResourceLimit(), Resources.createResource(160 * GB, 1));
     
     csConf.setFloat(CapacitySchedulerConfiguration.
         MAXIMUM_APPLICATION_MASTERS_RESOURCE_PERCENT, 0.2f);
     LeafQueue newA = new LeafQueue(csContext, A, root, null);
     a.reinitialize(newA, clusterResource);
     assertEquals(0.2f, a.getMaxAMResourcePerQueuePercent(), 1e-3f);
-    assertEquals(320, a.getMaximumActiveApplications());
+    assertEquals(a.getAMResourceLimit(), Resources.createResource(320 * GB, 1));
 
     Resource newClusterResource = Resources.createResource(100 * 20 * GB,
         100 * 32);
     a.updateClusterResource(newClusterResource);
     //  100 * 20 * 0.2 = 400
-    assertEquals(400, a.getMaximumActiveApplications());
+    assertEquals(a.getAMResourceLimit(), Resources.createResource(400 * GB, 1));
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c53420f5/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 2a49545..985609e 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
@@ -77,6 +77,7 @@ public class TestReservations {
       .getRecordFactory(null);
 
   RMContext rmContext;
+  RMContext spyRMContext;
   CapacityScheduler cs;
   // CapacitySchedulerConfiguration csConf;
   CapacitySchedulerContext csContext;
@@ -132,7 +133,10 @@ public class TestReservations {
     root = CapacityScheduler.parseQueue(csContext, csConf, null,
         CapacitySchedulerConfiguration.ROOT, queues, queues, TestUtils.spyHook);
 
-    cs.setRMContext(rmContext);
+    spyRMContext = spy(rmContext);
+    when(spyRMContext.getScheduler()).thenReturn(cs);
+    
+    cs.setRMContext(spyRMContext);
     cs.init(csConf);
     cs.start();
   }
@@ -212,14 +216,14 @@ public class TestReservations {
     final ApplicationAttemptId appAttemptId_0 = TestUtils
         .getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
 
     a.submitApplicationAttempt(app_0, user_0); 
 
     final ApplicationAttemptId appAttemptId_1 = TestUtils
         .getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0); 
 
     // Setup some nodes
@@ -361,14 +365,14 @@ public class TestReservations {
     final ApplicationAttemptId appAttemptId_0 = TestUtils
         .getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
 
     a.submitApplicationAttempt(app_0, user_0); 
 
     final ApplicationAttemptId appAttemptId_1 = TestUtils
         .getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0); 
 
     // Setup some nodes
@@ -506,14 +510,14 @@ public class TestReservations {
     final ApplicationAttemptId appAttemptId_0 = TestUtils
         .getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
 
     a.submitApplicationAttempt(app_0, user_0); 
 
     final ApplicationAttemptId appAttemptId_1 = TestUtils
         .getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0); 
 
     // Setup some nodes
@@ -618,7 +622,7 @@ public class TestReservations {
         .getMockApplicationAttemptId(0, 0);
     LeafQueue a = stubLeafQueue((LeafQueue) queues.get(A));
     FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
 
     String host_0 = "host_0";
     FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0,
@@ -685,7 +689,7 @@ public class TestReservations {
         .getMockApplicationAttemptId(0, 0);
     LeafQueue a = stubLeafQueue((LeafQueue) queues.get(A));
     FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
 
     String host_1 = "host_1";
     FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0,
@@ -742,14 +746,14 @@ public class TestReservations {
     final ApplicationAttemptId appAttemptId_0 = TestUtils
         .getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
 
     a.submitApplicationAttempt(app_0, user_0); 
 
     final ApplicationAttemptId appAttemptId_1 = TestUtils
         .getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0); 
 
     // Setup some nodes
@@ -916,14 +920,14 @@ public class TestReservations {
     final ApplicationAttemptId appAttemptId_0 = TestUtils
         .getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
 
     a.submitApplicationAttempt(app_0, user_0); 
 
     final ApplicationAttemptId appAttemptId_1 = TestUtils
         .getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0); 
 
     // Setup some nodes
@@ -1042,14 +1046,14 @@ public class TestReservations {
     final ApplicationAttemptId appAttemptId_0 = TestUtils
         .getMockApplicationAttemptId(0, 0);
     FiCaSchedulerApp app_0 = new FiCaSchedulerApp(appAttemptId_0, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
 
     a.submitApplicationAttempt(app_0, user_0); 
 
     final ApplicationAttemptId appAttemptId_1 = TestUtils
         .getMockApplicationAttemptId(1, 0);
     FiCaSchedulerApp app_1 = new FiCaSchedulerApp(appAttemptId_1, user_0, a,
-        mock(ActiveUsersManager.class), rmContext);
+        mock(ActiveUsersManager.class), spyRMContext);
     a.submitApplicationAttempt(app_1, user_0); 
 
     // Setup some nodes


[24/25] hadoop git commit: HADOOP-11261 Set custom endpoint for S3A. (Thomas Demoor via stevel)

Posted by zj...@apache.org.
HADOOP-11261 Set custom endpoint for S3A. (Thomas Demoor via stevel)


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

Branch: refs/heads/YARN-2928
Commit: 000ca83ea3aeb3687625c857bcc0762fd2887db2
Parents: 5d1cca3
Author: Steve Loughran <st...@apache.org>
Authored: Fri Jan 16 10:15:22 2015 +0000
Committer: Steve Loughran <st...@apache.org>
Committed: Fri Jan 16 10:15:51 2015 +0000

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  2 +
 .../org/apache/hadoop/fs/s3a/Constants.java     |  5 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java | 19 +++++
 .../hadoop/fs/s3a/TestS3AConfiguration.java     | 85 ++++++++++++++++++++
 4 files changed, 110 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/000ca83e/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 e1a6800..6896fe2 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -486,6 +486,8 @@ Release 2.7.0 - UNRELEASED
     HADOOP-8757. Metrics should disallow names with invalid characters
     (rchiang via rkanter)
 
+    HADOOP-11261 Set custom endpoint for S3A. (Thomas Demoor via stevel)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/000ca83e/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
index f1b5d3d..0232961 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java
@@ -28,7 +28,10 @@ public class Constants {
   // connect to s3 over ssl?
   public static final String SECURE_CONNECTIONS = "fs.s3a.connection.ssl.enabled";
   public static final boolean DEFAULT_SECURE_CONNECTIONS = true;
-  
+
+  //use a custom endpoint?
+  public static final String ENDPOINT = "fs.s3a.endpoint";
+
   // number of times we should retry errors
   public static final String MAX_ERROR_RETRIES = "fs.s3a.attempts.maximum";
   public static final int DEFAULT_MAX_ERROR_RETRIES = 10;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/000ca83e/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
index e6b1557..d8cf73f 100644
--- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
+++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
@@ -57,6 +57,7 @@ import com.amazonaws.services.s3.transfer.Upload;
 import com.amazonaws.event.ProgressListener;
 import com.amazonaws.event.ProgressEvent;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.lang.StringUtils;
 
 import org.apache.hadoop.conf.Configuration;
@@ -176,6 +177,16 @@ public class S3AFileSystem extends FileSystem {
       DEFAULT_SOCKET_TIMEOUT));
 
     s3 = new AmazonS3Client(credentials, awsConf);
+    String endPoint = conf.getTrimmed(ENDPOINT,"");
+    if (!endPoint.isEmpty()) {
+      try {
+        s3.setEndpoint(endPoint);
+      } catch (IllegalArgumentException e) {
+        String msg = "Incorrect endpoint: "  + e.getMessage();
+        LOG.error(msg);
+        throw new IllegalArgumentException(msg, e);
+      }
+    }
 
     maxKeys = conf.getInt(MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS);
     partSize = conf.getLong(MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE);
@@ -262,6 +273,14 @@ public class S3AFileSystem extends FileSystem {
     return uri;
   }
 
+  /**
+   * Returns the S3 client used by this filesystem.
+   * @return AmazonS3Client
+   */
+  @VisibleForTesting
+  AmazonS3Client getAmazonS3Client() {
+    return s3;
+  }
 
   public S3AFileSystem() {
     super();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/000ca83e/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
new file mode 100644
index 0000000..e4a14d0
--- /dev/null
+++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AConfiguration.java
@@ -0,0 +1,85 @@
+/**
+ * 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.fs.s3a;
+
+import com.amazonaws.services.s3.AmazonS3Client;
+import org.apache.commons.lang.StringUtils;
+import org.apache.hadoop.conf.Configuration;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class TestS3AConfiguration {
+  private Configuration conf;
+  private S3AFileSystem fs;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestS3AConfiguration.class);
+
+  private static final String TEST_ENDPOINT = "test.fs.s3a.endpoint";
+
+  @Rule
+  public Timeout testTimeout = new Timeout(30 * 60 * 1000);
+
+  /**
+   * Test if custom endpoint is picked up.
+   * <p/>
+   * The test expects TEST_ENDPOINT to be defined in the Configuration
+   * describing the endpoint of the bucket to which TEST_FS_S3A_NAME points
+   * (f.i. "s3-eu-west-1.amazonaws.com" if the bucket is located in Ireland).
+   * Evidently, the bucket has to be hosted in the region denoted by the
+   * endpoint for the test to succeed.
+   * <p/>
+   * More info and the list of endpoint identifiers:
+   * http://docs.aws.amazon.com/general/latest/gr/rande.html#s3_region
+   *
+   * @throws Exception
+   */
+  @Test
+  public void TestEndpoint() throws Exception {
+    conf = new Configuration();
+    String endpoint = conf.getTrimmed(TEST_ENDPOINT, "");
+    if (endpoint.isEmpty()) {
+      LOG.warn("Custom endpoint test skipped as " + TEST_ENDPOINT + "config " +
+          "setting was not detected");
+    } else {
+      conf.set(Constants.ENDPOINT, endpoint);
+      fs = S3ATestUtils.createTestFileSystem(conf);
+      AmazonS3Client s3 = fs.getAmazonS3Client();
+      String endPointRegion = "";
+      // Differentiate handling of "s3-" and "s3." based endpoint identifiers
+      String[] endpointParts = StringUtils.split(endpoint, '.');
+      if (endpointParts.length == 3) {
+        endPointRegion = endpointParts[0].substring(3);
+      } else if (endpointParts.length == 4) {
+        endPointRegion = endpointParts[1];
+      } else {
+        fail("Unexpected endpoint");
+      }
+      assertEquals("Endpoint config setting and bucket location differ: ",
+          endPointRegion, s3.getBucketLocation(fs.getUri().getHost()));
+    }
+  }
+}


[13/25] hadoop git commit: YARN-3005. [JDK7] Use switch statement for String instead of if-else statement in RegistrySecurity.java (Contributed by Kengo Seki)

Posted by zj...@apache.org.
YARN-3005. [JDK7] Use switch statement for String instead of if-else statement in RegistrySecurity.java (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/533e551e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/533e551e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/533e551e

Branch: refs/heads/YARN-2928
Commit: 533e551eb42af188535aeb0ab35f8ebf150a0da1
Parents: db51548
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Jan 15 21:27:39 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Jan 15 21:27:39 2015 +0900

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                        |  3 +++
 .../registry/client/impl/zk/RegistrySecurity.java      | 13 ++++++++-----
 2 files changed, 11 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/533e551e/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 5716b50..a87b41e 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -80,6 +80,9 @@ Release 2.7.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    YARN-3005. [JDK7] Use switch statement for String instead of if-else
+    statement in RegistrySecurity.java (Kengo Seki via aajisaka)
+
     YARN-2950. Change message to mandate, not suggest JS requirement on UI.
     (Dustin Cote via harsh)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/533e551e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
index d3c2d15..1c75e43 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java
@@ -192,14 +192,17 @@ public class RegistrySecurity extends AbstractService {
     String auth = conf.getTrimmed(KEY_REGISTRY_CLIENT_AUTH,
         REGISTRY_CLIENT_AUTH_ANONYMOUS);
 
-    // TODO JDK7 SWITCH
-    if (REGISTRY_CLIENT_AUTH_KERBEROS.equals(auth)) {
+    switch (auth) {
+    case REGISTRY_CLIENT_AUTH_KERBEROS:
       access = AccessPolicy.sasl;
-    } else if (REGISTRY_CLIENT_AUTH_DIGEST.equals(auth)) {
+      break;
+    case REGISTRY_CLIENT_AUTH_DIGEST:
       access = AccessPolicy.digest;
-    } else if (REGISTRY_CLIENT_AUTH_ANONYMOUS.equals(auth)) {
+      break;
+    case REGISTRY_CLIENT_AUTH_ANONYMOUS:
       access = AccessPolicy.anon;
-    } else {
+      break;
+    default:
       throw new ServiceStateException(E_UNKNOWN_AUTHENTICATION_MECHANISM
                                       + "\"" + auth + "\"");
     }


[22/25] hadoop git commit: HADOOP-11350. The size of header buffer of HttpServer is too small when HTTPS is enabled. Contributed by Benoy Antony.

Posted by zj...@apache.org.
HADOOP-11350. The size of header buffer of HttpServer is too small when HTTPS is enabled. Contributed by Benoy Antony.


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

Branch: refs/heads/YARN-2928
Commit: 3ab3a6498812c9fa0c53dae02ce696033062af87
Parents: 780a6bf
Author: Haohui Mai <wh...@apache.org>
Authored: Thu Jan 15 16:17:30 2015 -0800
Committer: Haohui Mai <wh...@apache.org>
Committed: Thu Jan 15 16:17:30 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../org/apache/hadoop/http/HttpServer2.java     |  1 +
 .../hadoop/http/HttpServerFunctionalTest.java   | 32 ++++++++++++++++++++
 .../org/apache/hadoop/http/TestHttpServer.java  | 24 +--------------
 .../apache/hadoop/http/TestSSLHttpServer.java   | 15 ++++++++-
 5 files changed, 51 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ab3a649/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 5cce74b..e1a6800 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -711,6 +711,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11462. TestSocketIOWithTimeout needs change for PowerPC platform.
     (Ayappan via cnauroth)
 
+    HADOOP-11350. The size of header buffer of HttpServer is too small when
+    HTTPS is enabled. (Benoy Antony via wheat9)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ab3a649/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
index 80e4005..80831e9 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java
@@ -279,6 +279,7 @@ public final class HttpServer2 implements FilterContainer {
           listener = HttpServer2.createDefaultChannelConnector();
         } else if ("https".equals(scheme)) {
           SslSocketConnector c = new SslSocketConnectorSecure();
+          c.setHeaderBufferSize(1024*64);
           c.setNeedClientAuth(needsClientAuth);
           c.setKeyPassword(keyPassword);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ab3a649/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
index ecf2d0f..4a4de41 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/HttpServerFunctionalTest.java
@@ -28,15 +28,32 @@ import org.apache.hadoop.http.HttpServer2.Builder;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
+import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URL;
 import java.net.MalformedURLException;
 
+import javax.servlet.ServletException;
+import javax.servlet.http.HttpServlet;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+
 /**
  * This is a base class for functional tests of the {@link HttpServer2}.
  * The methods are static for other classes to import statically.
  */
 public class HttpServerFunctionalTest extends Assert {
+  @SuppressWarnings("serial")
+  public static class LongHeaderServlet extends HttpServlet {
+    @Override
+    public void doGet(HttpServletRequest request,
+                      HttpServletResponse response
+    ) throws ServletException, IOException {
+      Assert.assertEquals(63 * 1024, request.getHeader("longheader").length());
+      response.setStatus(HttpServletResponse.SC_OK);
+    }
+  }
+
   /** JVM property for the webapp test dir : {@value} */
   public static final String TEST_BUILD_WEBAPPS = "test.build.webapps";
   /** expected location of the test.build.webapps dir: {@value} */
@@ -44,6 +61,7 @@ public class HttpServerFunctionalTest extends Assert {
   
   /** name of the test webapp: {@value} */
   private static final String TEST = "test";
+  protected static URL baseUrl;
 
   /**
    * Create but do not start the test webapp server. The test webapp dir is
@@ -227,4 +245,18 @@ public class HttpServerFunctionalTest extends Assert {
     }
     return out.toString();
   }
+
+  /**
+   *  Test that verifies headers can be up to 64K long.
+   *  The test adds a 63K header leaving 1K for other headers.
+   *  This is because the header buffer setting is for ALL headers,
+   *  names and values included. */
+  protected void testLongHeader(HttpURLConnection conn) throws IOException {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0 ; i < 63 * 1024; i++) {
+      sb.append("a");
+    }
+    conn.setRequestProperty("longheader", sb.toString());
+    assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ab3a649/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
index a2034f3..5b202da 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServer.java
@@ -67,7 +67,6 @@ import java.util.concurrent.Executors;
 public class TestHttpServer extends HttpServerFunctionalTest {
   static final Log LOG = LogFactory.getLog(TestHttpServer.class);
   private static HttpServer2 server;
-  private static URL baseUrl;
   private static final int MAX_THREADS = 10;
   
   @SuppressWarnings("serial")
@@ -121,17 +120,6 @@ public class TestHttpServer extends HttpServerFunctionalTest {
   }
 
   @SuppressWarnings("serial")
-  public static class LongHeaderServlet extends HttpServlet {
-    @Override
-    public void doGet(HttpServletRequest request,
-                      HttpServletResponse response
-    ) throws ServletException, IOException {
-      Assert.assertEquals(63 * 1024, request.getHeader("longheader").length());
-      response.setStatus(HttpServletResponse.SC_OK);
-    }
-  }
-
-  @SuppressWarnings("serial")
   public static class HtmlContentServlet extends HttpServlet {
     @Override
     public void doGet(HttpServletRequest request, 
@@ -210,20 +198,10 @@ public class TestHttpServer extends HttpServerFunctionalTest {
                  readOutput(new URL(baseUrl, "/echomap?a=b&c<=d&a=>")));
   }
 
-  /** 
-   *  Test that verifies headers can be up to 64K long. 
-   *  The test adds a 63K header leaving 1K for other headers.
-   *  This is because the header buffer setting is for ALL headers,
-   *  names and values included. */
   @Test public void testLongHeader() throws Exception {
     URL url = new URL(baseUrl, "/longheader");
     HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    StringBuilder sb = new StringBuilder();
-    for (int i = 0 ; i < 63 * 1024; i++) {
-      sb.append("a");
-    }
-    conn.setRequestProperty("longheader", sb.toString());
-    assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
+    testLongHeader(conn);
   }
 
   @Test public void testContentTypes() throws Exception {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3ab3a649/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
index 3d5d8b6..70fea87 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestSSLHttpServer.java
@@ -49,7 +49,6 @@ public class TestSSLHttpServer extends HttpServerFunctionalTest {
   private static final Log LOG = LogFactory.getLog(TestSSLHttpServer.class);
   private static Configuration conf;
   private static HttpServer2 server;
-  private static URL baseUrl;
   private static String keystoresDir;
   private static String sslConfDir;
   private static SSLFactory clientSslFactory;
@@ -85,6 +84,7 @@ public class TestSSLHttpServer extends HttpServerFunctionalTest {
             sslConf.get("ssl.server.truststore.password"),
             sslConf.get("ssl.server.truststore.type", "jks")).build();
     server.addServlet("echo", "/echo", TestHttpServer.EchoServlet.class);
+    server.addServlet("longheader", "/longheader", LongHeaderServlet.class);
     server.start();
     baseUrl = new URL("https://"
         + NetUtils.getHostPortString(server.getConnectorAddress(0)));
@@ -106,6 +106,19 @@ public class TestSSLHttpServer extends HttpServerFunctionalTest {
         "/echo?a=b&c<=d&e=>")));
   }
 
+  /**
+   *  Test that verifies headers can be up to 64K long.
+   *  The test adds a 63K header leaving 1K for other headers.
+   *  This is because the header buffer setting is for ALL headers,
+   *  names and values included. */
+  @Test
+  public void testLongHeader() throws Exception {
+    URL url = new URL(baseUrl, "/longheader");
+    HttpsURLConnection conn = (HttpsURLConnection) url.openConnection();
+    conn.setSSLSocketFactory(clientSslFactory.createSSLSocketFactory());
+    testLongHeader(conn);
+  }
+
   private static String readOut(URL url) throws Exception {
     HttpsURLConnection conn = (HttpsURLConnection) url.openConnection();
     conn.setSSLSocketFactory(clientSslFactory.createSSLSocketFactory());


[18/25] hadoop git commit: YARN-2861. Fixed Timeline DT secret manager to not reuse RM's configs. Contributed by Zhijie Shen

Posted by zj...@apache.org.
YARN-2861. Fixed Timeline DT secret manager to not reuse RM's configs. 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/9e33116d
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9e33116d
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9e33116d

Branch: refs/heads/YARN-2928
Commit: 9e33116d1d8944a393937337b3963e192b9c74d1
Parents: a6efbb2
Author: Jian He <ji...@apache.org>
Authored: Thu Jan 15 11:07:43 2015 -0800
Committer: Jian He <ji...@apache.org>
Committed: Thu Jan 15 11:07:43 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |  3 +++
 .../hadoop/yarn/conf/YarnConfiguration.java     | 28 +++++++++++++++-----
 ...lineDelegationTokenSecretManagerService.java | 12 ++++-----
 .../resourcemanager/RMSecretManagerService.java | 12 ++++-----
 4 files changed, 36 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e33116d/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a87b41e..9173deb 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -371,6 +371,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2637. Fixed max-am-resource-percent calculation in CapacityScheduler
     when activating applications. (Craig Welch via jianhe)
 
+    YARN-2861. Fixed Timeline DT secret manager to not reuse RM's configs.
+    (Zhijie Shen via jianhe)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e33116d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 0e92194..9ab5298 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -356,18 +356,18 @@ public class YarnConfiguration extends Configuration {
   public static final int DEFAULT_RM_SYSTEM_METRICS_PUBLISHER_DISPATCHER_POOL_SIZE =
       10;
 
-  //Delegation token related keys
-  public static final String  DELEGATION_KEY_UPDATE_INTERVAL_KEY = 
+  //RM delegation token related keys
+  public static final String RM_DELEGATION_KEY_UPDATE_INTERVAL_KEY =
     RM_PREFIX + "delegation.key.update-interval";
-  public static final long    DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT = 
+  public static final long RM_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT =
     24*60*60*1000; // 1 day
-  public static final String  DELEGATION_TOKEN_RENEW_INTERVAL_KEY = 
+  public static final String RM_DELEGATION_TOKEN_RENEW_INTERVAL_KEY =
     RM_PREFIX + "delegation.token.renew-interval";
-  public static final long    DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT = 
+  public static final long RM_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT =
     24*60*60*1000;  // 1 day
-  public static final String  DELEGATION_TOKEN_MAX_LIFETIME_KEY = 
+  public static final String RM_DELEGATION_TOKEN_MAX_LIFETIME_KEY =
      RM_PREFIX + "delegation.token.max-lifetime";
-  public static final long    DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT = 
+  public static final long RM_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT =
     7*24*60*60*1000; // 7 days
   
   public static final String RECOVERY_ENABLED = RM_PREFIX + "recovery.enabled";
@@ -1382,6 +1382,20 @@ public class YarnConfiguration extends Configuration {
   public static final String TIMELINE_SERVICE_LEVELDB_STATE_STORE_PATH =
       TIMELINE_SERVICE_LEVELDB_STATE_STORE_PREFIX + "path";
 
+  // Timeline delegation token related keys
+  public static final String  TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL =
+      TIMELINE_SERVICE_PREFIX + "delegation.key.update-interval";
+  public static final long    DEFAULT_TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL =
+      24*60*60*1000; // 1 day
+  public static final String  TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL =
+      TIMELINE_SERVICE_PREFIX + "delegation.token.renew-interval";
+  public static final long    DEFAULT_TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL =
+      24*60*60*1000;  // 1 day
+  public static final String  TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME =
+      TIMELINE_SERVICE_PREFIX + "delegation.token.max-lifetime";
+  public static final long    DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME =
+      7*24*60*60*1000; // 7 days
+
   // ///////////////////////////////
   // Shared Cache Configs
   // ///////////////////////////////

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e33116d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
index 261a335..c940eea 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/timeline/security/TimelineDelegationTokenSecretManagerService.java
@@ -60,14 +60,14 @@ public class TimelineDelegationTokenSecretManagerService extends
     }
 
     long secretKeyInterval =
-        conf.getLong(YarnConfiguration.DELEGATION_KEY_UPDATE_INTERVAL_KEY,
-            YarnConfiguration.DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT);
+        conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL,
+            YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_KEY_UPDATE_INTERVAL);
     long tokenMaxLifetime =
-        conf.getLong(YarnConfiguration.DELEGATION_TOKEN_MAX_LIFETIME_KEY,
-            YarnConfiguration.DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT);
+        conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME,
+            YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_MAX_LIFETIME);
     long tokenRenewInterval =
-        conf.getLong(YarnConfiguration.DELEGATION_TOKEN_RENEW_INTERVAL_KEY,
-            YarnConfiguration.DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT);
+        conf.getLong(YarnConfiguration.TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL,
+            YarnConfiguration.DEFAULT_TIMELINE_DELEGATION_TOKEN_RENEW_INTERVAL);
     secretManager = new TimelineDelegationTokenSecretManager(secretKeyInterval,
         tokenMaxLifetime, tokenRenewInterval, 3600000, stateStore);
     super.init(conf);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9e33116d/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMSecretManagerService.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/RMSecretManagerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMSecretManagerService.java
index d0d7d16..f7fb7e6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMSecretManagerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMSecretManagerService.java
@@ -127,14 +127,14 @@ public class RMSecretManagerService extends AbstractService {
   protected RMDelegationTokenSecretManager createRMDelegationTokenSecretManager(
       Configuration conf, RMContext rmContext) {
     long secretKeyInterval =
-        conf.getLong(YarnConfiguration.DELEGATION_KEY_UPDATE_INTERVAL_KEY,
-            YarnConfiguration.DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT);
+        conf.getLong(YarnConfiguration.RM_DELEGATION_KEY_UPDATE_INTERVAL_KEY,
+            YarnConfiguration.RM_DELEGATION_KEY_UPDATE_INTERVAL_DEFAULT);
     long tokenMaxLifetime =
-        conf.getLong(YarnConfiguration.DELEGATION_TOKEN_MAX_LIFETIME_KEY,
-            YarnConfiguration.DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT);
+        conf.getLong(YarnConfiguration.RM_DELEGATION_TOKEN_MAX_LIFETIME_KEY,
+            YarnConfiguration.RM_DELEGATION_TOKEN_MAX_LIFETIME_DEFAULT);
     long tokenRenewInterval =
-        conf.getLong(YarnConfiguration.DELEGATION_TOKEN_RENEW_INTERVAL_KEY,
-            YarnConfiguration.DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT);
+        conf.getLong(YarnConfiguration.RM_DELEGATION_TOKEN_RENEW_INTERVAL_KEY,
+            YarnConfiguration.RM_DELEGATION_TOKEN_RENEW_INTERVAL_DEFAULT);
 
     return new RMDelegationTokenSecretManager(secretKeyInterval,
         tokenMaxLifetime, tokenRenewInterval, 3600000, rmContext);


[05/25] hadoop git commit: YARN-2807. Option "--forceactive" not works as described in usage of "yarn rmadmin -transitionToActive". Contributed by Masatake Iwasaki

Posted by zj...@apache.org.
YARN-2807. Option "--forceactive" not works as described in usage of
"yarn rmadmin -transitionToActive". Contributed by Masatake Iwasaki


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

Branch: refs/heads/YARN-2928
Commit: d15cbae73c7ae22d5d60d8cba16cba565e8e8b20
Parents: d336d13
Author: Xuan <xg...@apache.org>
Authored: Wed Jan 14 11:04:03 2015 -0800
Committer: Xuan <xg...@apache.org>
Committed: Wed Jan 14 11:04:03 2015 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/hadoop/ha/HAAdmin.java | 10 ++--
 hadoop-yarn-project/CHANGES.txt                 |  3 ++
 .../src/site/apt/ResourceManagerHA.apt.vm       |  1 +
 .../src/site/apt/YarnCommands.apt.vm            | 52 ++++++++++++++------
 4 files changed, 46 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d15cbae7/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
index a6c4a42..f72df77 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java
@@ -69,15 +69,15 @@ public abstract class HAAdmin extends Configured implements Tool {
   protected final static Map<String, UsageInfo> USAGE =
     ImmutableMap.<String, UsageInfo>builder()
     .put("-transitionToActive",
-        new UsageInfo("<serviceId> [--"+FORCEACTIVE+"]", "Transitions the service into Active state"))
+        new UsageInfo("[--"+FORCEACTIVE+"] <serviceId>", "Transitions the service into Active state"))
     .put("-transitionToStandby",
         new UsageInfo("<serviceId>", "Transitions the service into Standby state"))
     .put("-failover",
         new UsageInfo("[--"+FORCEFENCE+"] [--"+FORCEACTIVE+"] <serviceId> <serviceId>",
             "Failover from the first service to the second.\n" +
-            "Unconditionally fence services if the "+FORCEFENCE+" option is used.\n" +
+            "Unconditionally fence services if the --"+FORCEFENCE+" option is used.\n" +
             "Try to failover to the target service even if it is not ready if the " + 
-            FORCEACTIVE + " option is used."))
+            "--" + FORCEACTIVE + " option is used."))
     .put("-getServiceState",
         new UsageInfo("<serviceId>", "Returns the state of the service"))
     .put("-checkHealth",
@@ -228,7 +228,7 @@ public abstract class HAAdmin extends Configured implements Tool {
             "Refusing to manually manage HA state, since it may cause\n" +
             "a split-brain scenario or other incorrect state.\n" +
             "If you are very sure you know what you are doing, please \n" +
-            "specify the " + FORCEMANUAL + " flag.");
+            "specify the --" + FORCEMANUAL + " flag.");
         return false;
       } else {
         LOG.warn("Proceeding with manual HA state management even though\n" +
@@ -462,7 +462,7 @@ public abstract class HAAdmin extends Configured implements Tool {
   
   private boolean confirmForceManual() throws IOException {
      return ToolRunner.confirmPrompt(
-        "You have specified the " + FORCEMANUAL + " flag. This flag is " +
+        "You have specified the --" + FORCEMANUAL + " flag. This flag is " +
         "dangerous, as it can induce a split-brain scenario that WILL " +
         "CORRUPT your HDFS namespace, possibly irrecoverably.\n" +
         "\n" +

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d15cbae7/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 1d2acc1..a398347 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -186,6 +186,9 @@ Release 2.7.0 - UNRELEASED
     YARN-3019. Make work-preserving-recovery the default mechanism for RM 
     recovery. (Jian He via junping_du)
 
+    YARN-2807. Option "--forceactive" not works as described in usage of
+    "yarn rmadmin -transitionToActive". (Masatake Iwasaki via xgong)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d15cbae7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerHA.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerHA.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerHA.apt.vm
index 8cfdd79..0346cda 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerHA.apt.vm
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ResourceManagerHA.apt.vm
@@ -208,6 +208,7 @@ ResourceManager High Availability
 +---+
 
    If automatic failover is enabled, you can not use manual transition command.
+   Though you can override this by --forcemanual flag, you need caution.
 
 +---+
  $ yarn rmadmin -transitionToStandby rm1

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d15cbae7/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YarnCommands.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YarnCommands.apt.vm b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YarnCommands.apt.vm
index 2de901b..6333d70 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YarnCommands.apt.vm
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/YarnCommands.apt.vm
@@ -246,13 +246,19 @@ YARN Commands
   Usage:
 
 ----
-  yarn rmadmin [-refreshQueues] [-refreshNodes] [-refreshUserToGroupsMapping]
-                      [-refreshSuperUserGroupsConfiguration] [-refreshAdminAcls]
-                      [-refreshServiceAcl] [-getGroups [username]] [-help [cmd]]
-                      [-transitionToActive <serviceId>]
-                      [-transitionToStandby <serviceId>]
-                      [-getServiceState <serviceId>]
-                      [-checkHealth <serviceId>]
+  yarn rmadmin [-refreshQueues]
+               [-refreshNodes]
+               [-refreshUserToGroupsMapping] 
+               [-refreshSuperUserGroupsConfiguration]
+               [-refreshAdminAcls] 
+               [-refreshServiceAcl]
+               [-getGroups [username]]
+               [-transitionToActive [--forceactive] [--forcemanual] <serviceId>]
+               [-transitionToStandby [--forcemanual] <serviceId>]
+               [-failover [--forcefence] [--forceactive] <serviceId1> <serviceId2>]
+               [-getServiceState <serviceId>]
+               [-checkHealth <serviceId>]
+               [-help [cmd]]
 ----
 
 *---------------+--------------+
@@ -277,14 +283,26 @@ YARN Commands
 *---------------+--------------+
 | -getGroups [username] | Get groups the specified user belongs to.
 *---------------+--------------+
-| -help [cmd] | Displays help for the given command or all commands if none is
-|             | specified.
-*---------------+--------------+
-| -transitionToActive \<serviceId\> | Transitions the service into Active
-|                                   | state.
-*---------------+--------------+
-| -transitionToStandby \<serviceId\> | Transitions the service into Standby
-|                                    | state.
+| -transitionToActive [--forceactive] [--forcemanual] \<serviceId\> |
+|               | Transitions the service into Active state.
+|               | Try to make the target active
+|               | without checking that there is no active node
+|               | if the --forceactive option is used.
+|               | This command can not be used if automatic failover is enabled.
+|               | Though you can override this by --forcemanual option,
+|               | you need caution.
+*---------------+--------------+
+| -transitionToStandby [--forcemanual] \<serviceId\> |
+|               | Transitions the service into Standby state.
+|               | This command can not be used if automatic failover is enabled.
+|               | Though you can override this by --forcemanual option,
+|               | you need caution.
+*---------------+--------------+
+| -failover [--forceactive] \<serviceId1\> \<serviceId2\> |
+|               | Initiate a failover from serviceId1 to serviceId2.
+|               | Try to failover to the target service even if it is not ready
+|               | if the --forceactive option is used.
+|               | This command can not be used if automatic failover is enabled.
 *---------------+--------------+
 | -getServiceState \<serviceId\> | Returns the state of the service.
 *---------------+--------------+
@@ -292,6 +310,10 @@ YARN Commands
 |                            | check. The RMAdmin tool will exit with a
 |                            | non-zero exit code if the check fails.
 *---------------+--------------+
+| -help [cmd] | Displays help for the given command or all commands if none is
+|             | specified.
+*---------------+--------------+
+
 
   Runs ResourceManager admin client
 


[03/25] hadoop git commit: YARN-3019. Make work-preserving-recovery the default mechanism for RM recovery. (Contributed by Jian He)

Posted by zj...@apache.org.
YARN-3019. Make work-preserving-recovery the default mechanism for RM recovery. (Contributed by Jian He)


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

Branch: refs/heads/YARN-2928
Commit: f92e5038000a012229c304bc6e5281411eff2883
Parents: c53420f
Author: Junping Du <ju...@apache.org>
Authored: Tue Jan 13 18:20:57 2015 -0800
Committer: Junping Du <ju...@apache.org>
Committed: Tue Jan 13 18:20:57 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f92e5038/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index bbdfcb5..1d2acc1 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -183,6 +183,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2643. Don't create a new DominantResourceCalculator on every
     FairScheduler.allocate call. (kasha via rkanter)
 
+    YARN-3019. Make work-preserving-recovery the default mechanism for RM 
+    recovery. (Jian He via junping_du)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f92e5038/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
index 3273b47..0e92194 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java
@@ -378,7 +378,7 @@ public class YarnConfiguration extends Configuration {
       + "work-preserving-recovery.enabled";
   @Private
   public static final boolean DEFAULT_RM_WORK_PRESERVING_RECOVERY_ENABLED =
-      false;
+      true;
 
   public static final String RM_WORK_PRESERVING_RECOVERY_SCHEDULING_WAIT_MS =
       RM_PREFIX + "work-preserving-recovery.scheduling-wait-ms";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f92e5038/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
index 7dea2c3..349e57b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml
@@ -271,7 +271,7 @@
     to YARN for experimenting the feature.
     </description>
     <name>yarn.resourcemanager.work-preserving-recovery.enabled</name>
-    <value>false</value>
+    <value>true</value>
   </property>
 
   <property>


[21/25] hadoop git commit: HDFS-7457. DatanodeID generates excessive garbage. Contributed by Daryn Sharp.

Posted by zj...@apache.org.
HDFS-7457. DatanodeID generates excessive garbage. Contributed by Daryn Sharp.


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

Branch: refs/heads/YARN-2928
Commit: 780a6bf14562fd9d1070a7c8e756fa1c3bc65d32
Parents: 44eed6c
Author: Kihwal Lee <ki...@apache.org>
Authored: Thu Jan 15 16:44:11 2015 -0600
Committer: Kihwal Lee <ki...@apache.org>
Committed: Thu Jan 15 16:44:11 2015 -0600

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 ++
 .../apache/hadoop/hdfs/protocol/DatanodeID.java | 23 ++++++++++++++++++--
 2 files changed, 23 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/780a6bf1/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 823b3d1..2d4c634 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -515,6 +515,8 @@ Release 2.7.0 - UNRELEASED
     particular namenode in a federated cluster with multiple namenodes
     can be specified in the path parameter.  (szetszwo)
 
+    HDFS-7457. DatanodeID generates excessive garbage. (daryn via kihwal)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/780a6bf1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
index c781e5b..779e3b9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java
@@ -46,6 +46,8 @@ public class DatanodeID implements Comparable<DatanodeID> {
   private int infoPort;      // info server port
   private int infoSecurePort; // info server port
   private int ipcPort;       // IPC server port
+  private String xferAddr;
+  private int hashCode = -1;
 
   /**
    * UUID identifying a given datanode. For upgraded Datanodes this is the
@@ -86,10 +88,12 @@ public class DatanodeID implements Comparable<DatanodeID> {
     this.infoPort = infoPort;
     this.infoSecurePort = infoSecurePort;
     this.ipcPort = ipcPort;
+    updateXferAddrAndInvalidateHashCode();
   }
   
   public void setIpAddr(String ipAddr) {
     this.ipAddr = ipAddr;
+    updateXferAddrAndInvalidateHashCode();
   }
 
   public void setPeerHostName(String peerHostName) {
@@ -106,6 +110,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
   @VisibleForTesting
   public void setDatanodeUuidForTesting(String datanodeUuid) {
     this.datanodeUuid = datanodeUuid;
+    updateXferAddrAndInvalidateHashCode();
   }
 
   private String checkDatanodeUuid(String uuid) {
@@ -141,7 +146,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
    * @return IP:xferPort string
    */
   public String getXferAddr() {
-    return ipAddr + ":" + xferPort;
+    return xferAddr;
   }
 
   /**
@@ -237,7 +242,11 @@ public class DatanodeID implements Comparable<DatanodeID> {
   
   @Override
   public int hashCode() {
-    return getXferAddr().hashCode()^ datanodeUuid.hashCode();
+    if (hashCode == -1) {
+      int newHashCode = xferAddr.hashCode() ^ datanodeUuid.hashCode();
+      hashCode = newHashCode & Integer.MAX_VALUE;
+    }
+    return hashCode;
   }
   
   @Override
@@ -257,6 +266,7 @@ public class DatanodeID implements Comparable<DatanodeID> {
     infoPort = nodeReg.getInfoPort();
     infoSecurePort = nodeReg.getInfoSecurePort();
     ipcPort = nodeReg.getIpcPort();
+    updateXferAddrAndInvalidateHashCode();
   }
     
   /**
@@ -269,4 +279,13 @@ public class DatanodeID implements Comparable<DatanodeID> {
   public int compareTo(DatanodeID that) {
     return getXferAddr().compareTo(that.getXferAddr());
   }
+
+  // NOTE: mutable hash codes are dangerous, however this class chooses to
+  // use them.  this method must be called when a value mutates that is used
+  // to compute the hash, equality, or comparison of instances.
+  private void updateXferAddrAndInvalidateHashCode() {
+    xferAddr = ipAddr + ":" + xferPort;
+    // can't compute new hash yet because uuid might still null...
+    hashCode = -1;
+  }
 }


[04/25] hadoop git commit: MAPREDUCE-6210. Use getApplicationAttemptId() instead of getApplicationID() for logging AttemptId in RMContainerAllocator.java (Contributed by Leitao Guo)

Posted by zj...@apache.org.
MAPREDUCE-6210. Use getApplicationAttemptId() instead of getApplicationID() for logging AttemptId in RMContainerAllocator.java (Contributed by Leitao Guo)


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

Branch: refs/heads/YARN-2928
Commit: d336d136785ef1e49e0a110a1b9f0d1824829877
Parents: f92e503
Author: Akira Ajisaka <aa...@apache.org>
Authored: Wed Jan 14 17:38:35 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Wed Jan 14 17:38:35 2015 +0900

----------------------------------------------------------------------
 hadoop-mapreduce-project/CHANGES.txt                              | 3 +++
 .../apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java   | 2 +-
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d336d136/hadoop-mapreduce-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 8055446..a7379ec 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -268,6 +268,9 @@ Release 2.7.0 - UNRELEASED
 
   BUG FIXES
 
+    MAPREDUCE-6210. Use getApplicationAttemptId() instead of getApplicationId()
+    for logging AttemptId in RMContainerAllocator.java (Leitao Guo via aajisaka)
+
     MAPREDUCE-6177. Minor typo in the EncryptedShuffle document about
     ssl-client.xml (Yangping Wu via harsh)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d336d136/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
----------------------------------------------------------------------
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index e068997..0a4f2f3 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -684,7 +684,7 @@ public class RMContainerAllocator extends RMContainerRequestor
         JobEventType.JOB_AM_REBOOT));
       throw new YarnRuntimeException(
         "Resource Manager doesn't recognize AttemptId: "
-            + this.getContext().getApplicationID(), e);
+            + this.getContext().getApplicationAttemptId(), e);
     } catch (ApplicationMasterNotRegisteredException e) {
       LOG.info("ApplicationMaster is out of sync with ResourceManager,"
           + " hence resync and send outstanding requests.");


[20/25] hadoop git commit: HDFS-7615. Remove longReadLock. Contributed by Kihwal Lee.

Posted by zj...@apache.org.
HDFS-7615. Remove longReadLock. Contributed by Kihwal Lee.


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

Branch: refs/heads/YARN-2928
Commit: 44eed6cbc97649c15177f9b36f6b119cc1900f7a
Parents: b6ff9c0
Author: Kihwal Lee <ki...@apache.org>
Authored: Thu Jan 15 14:58:52 2015 -0600
Committer: Kihwal Lee <ki...@apache.org>
Committed: Thu Jan 15 16:39:18 2015 -0600

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  2 ++
 .../hdfs/server/namenode/FSNamesystem.java      | 34 +-------------------
 .../hdfs/server/namenode/FSNamesystemLock.java  | 22 -------------
 .../org/apache/hadoop/hdfs/util/RwLock.java     |  9 ------
 4 files changed, 3 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/44eed6cb/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 c3d32b8..823b3d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -520,6 +520,8 @@ Release 2.7.0 - UNRELEASED
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
     (Vinayakumar B via wheat9)
 
+    HDFS-7615. Remove longReadLock (kihwal)
+
   BUG FIXES
 
     HDFS-6741. Improve permission denied message when

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44eed6cb/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 e7bf8e9..9d2fe6b 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
@@ -1465,47 +1465,20 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     this.fsLock.readLock().lock();
   }
   @Override
-  public void longReadLockInterruptibly() throws InterruptedException {
-    this.fsLock.longReadLock().lockInterruptibly();
-    try {
-      this.fsLock.readLock().lockInterruptibly();
-    } catch (InterruptedException ie) {
-      // In the event we're interrupted while getting the normal FSNS read lock,
-      // release the long read lock.
-      this.fsLock.longReadLock().unlock();
-      throw ie;
-    }
-  }
-  @Override
-  public void longReadUnlock() {
-    this.fsLock.readLock().unlock();
-    this.fsLock.longReadLock().unlock();
-  }
-  @Override
   public void readUnlock() {
     this.fsLock.readLock().unlock();
   }
   @Override
   public void writeLock() {
-    this.fsLock.longReadLock().lock();
     this.fsLock.writeLock().lock();
   }
   @Override
   public void writeLockInterruptibly() throws InterruptedException {
-    this.fsLock.longReadLock().lockInterruptibly();
-    try {
-      this.fsLock.writeLock().lockInterruptibly();
-    } catch (InterruptedException ie) {
-      // In the event we're interrupted while getting the normal FSNS write
-      // lock, release the long read lock.
-      this.fsLock.longReadLock().unlock();
-      throw ie;
-    }
+    this.fsLock.writeLock().lockInterruptibly();
   }
   @Override
   public void writeUnlock() {
     this.fsLock.writeLock().unlock();
-    this.fsLock.longReadLock().unlock();
   }
   @Override
   public boolean hasWriteLock() {
@@ -7138,11 +7111,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
   
   @VisibleForTesting
-  public ReentrantLock getLongReadLockForTests() {
-    return fsLock.longReadLock;
-  }
-
-  @VisibleForTesting
   public ReentrantLock getCpLockForTests() {
     return cpLock;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44eed6cb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
index f031284..7e820d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
@@ -34,24 +34,6 @@ class FSNamesystemLock implements ReadWriteLock {
   @VisibleForTesting
   protected ReentrantReadWriteLock coarseLock;
   
-  /**
-   * When locking the FSNS for a read that may take a long time, we take this
-   * lock before taking the regular FSNS read lock. All writers also take this
-   * lock before taking the FSNS write lock. Regular (short) readers do not
-   * take this lock at all, instead relying solely on the synchronization of the
-   * regular FSNS lock.
-   * 
-   * This scheme ensures that:
-   * 1) In the case of normal (fast) ops, readers proceed concurrently and
-   *    writers are not starved.
-   * 2) In the case of long read ops, short reads are allowed to proceed
-   *    concurrently during the duration of the long read.
-   * 
-   * See HDFS-5064 for more context.
-   */
-  @VisibleForTesting
-  protected final ReentrantLock longReadLock = new ReentrantLock(true);
-  
   FSNamesystemLock(boolean fair) {
     this.coarseLock = new ReentrantReadWriteLock(fair);
   }
@@ -66,10 +48,6 @@ class FSNamesystemLock implements ReadWriteLock {
     return coarseLock.writeLock();
   }
 
-  public Lock longReadLock() {
-    return longReadLock;
-  }
-  
   public int getReadHoldCount() {
     return coarseLock.getReadHoldCount();
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/44eed6cb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java
index 2792460..e36f0f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java
@@ -22,15 +22,6 @@ public interface RwLock {
   /** Acquire read lock. */
   public void readLock();
   
-  /**
-   * Acquire the long read lock, unless interrupted while waiting. The long
-   * read lock should also serve to block all concurrent writers.
-   **/
-  void longReadLockInterruptibly() throws InterruptedException;
-  
-  /** Release the long read lock. */
-  public void longReadUnlock();
-
   /** Release read lock. */
   public void readUnlock();
 


[25/25] hadoop git commit: Merge remote-tracking branch 'apache/trunk' into YARN-2928

Posted by zj...@apache.org.
Merge remote-tracking branch 'apache/trunk' into YARN-2928


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

Branch: refs/heads/YARN-2928
Commit: 630f1cd4470abbeccfda39ddc1a8777e966ed7da
Parents: fe7d0e7 000ca83
Author: Zhijie Shen <zj...@apache.org>
Authored: Fri Jan 16 10:30:06 2015 -0800
Committer: Zhijie Shen <zj...@apache.org>
Committed: Fri Jan 16 10:30:06 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  15 +
 .../hadoop-common/src/main/bin/hadoop           |   8 +-
 .../src/main/bin/hadoop-functions.sh            |  11 +
 .../hadoop/crypto/key/JavaKeyStoreProvider.java |   5 +-
 .../java/org/apache/hadoop/fs/HardLink.java     | 313 +-----------
 .../java/org/apache/hadoop/fs/shell/Stat.java   |  33 +-
 .../main/java/org/apache/hadoop/ha/HAAdmin.java |  10 +-
 .../org/apache/hadoop/http/HttpServer2.java     |   1 +
 .../hadoop/metrics2/lib/MetricsRegistry.java    |  14 +
 .../src/site/apt/FileSystemShell.apt.vm         |  11 +-
 .../crypto/key/TestKeyProviderFactory.java      |  48 ++
 .../java/org/apache/hadoop/fs/TestHardLink.java |  91 +---
 .../hadoop/http/HttpServerFunctionalTest.java   |  32 ++
 .../org/apache/hadoop/http/TestHttpServer.java  |  24 +-
 .../apache/hadoop/http/TestSSLHttpServer.java   |  15 +-
 .../metrics2/lib/TestMetricsRegistry.java       |  42 ++
 .../src/test/resources/hdfs7067.keystore        | Bin 0 -> 6067 bytes
 .../src/test/resources/testConf.xml             |  24 +-
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  23 +
 .../hadoop-hdfs/src/main/bin/hdfs               |   4 +-
 .../hadoop-hdfs/src/main/bin/hdfs.cmd           |  14 +-
 .../hadoop/hdfs/BlockStorageLocationUtil.java   |  19 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 407 +++++++++++++---
 .../hadoop/hdfs/DFSInotifyEventInputStream.java | 144 +++---
 .../hadoop/hdfs/protocol/BlockListAsLongs.java  |  60 ++-
 .../hdfs/protocol/CacheDirectiveIterator.java   |  10 +-
 .../hadoop/hdfs/protocol/CachePoolIterator.java |  14 +-
 .../apache/hadoop/hdfs/protocol/DatanodeID.java |  23 +-
 .../hdfs/protocol/EncryptionZoneIterator.java   |  15 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  34 +-
 .../hdfs/server/namenode/FSNamesystemLock.java  |  22 -
 .../org/apache/hadoop/hdfs/tools/DFSck.java     |  32 +-
 .../org/apache/hadoop/hdfs/util/RwLock.java     |   9 -
 .../hadoop-hdfs/src/site/apt/Federation.apt.vm  | 158 +++----
 .../src/site/apt/HDFSCommands.apt.vm            | 471 +++++++++++++++----
 .../site/apt/HDFSHighAvailabilityWithNFS.apt.vm |  70 +--
 .../site/apt/HDFSHighAvailabilityWithQJM.apt.vm |  40 +-
 .../src/site/apt/HdfsNfsGateway.apt.vm          | 108 ++---
 .../hadoop/fs/TestEnhancedByteBufferAccess.java |  66 +--
 .../server/datanode/SimulatedFSDataset.java     |   4 +-
 .../TestBlockHasMultipleReplicasOnSameDN.java   |   2 +-
 .../server/namenode/NNThroughputBenchmark.java  |   2 +-
 .../server/namenode/TestCacheDirectives.java    |   3 +-
 .../namenode/TestFsckWithMultipleNameNodes.java | 141 ++++++
 hadoop-mapreduce-project/CHANGES.txt            |   3 +
 .../v2/app/rm/RMContainerAllocator.java         |   2 +-
 .../org/apache/hadoop/fs/s3a/Constants.java     |   5 +-
 .../org/apache/hadoop/fs/s3a/S3AFileSystem.java |  19 +
 .../hadoop/fs/s3a/TestS3AConfiguration.java     |  85 ++++
 hadoop-yarn-project/CHANGES.txt                 |  21 +
 .../dev-support/findbugs-exclude.xml            |   1 +
 .../hadoop/yarn/conf/YarnConfiguration.java     |  30 +-
 .../yarn/client/api/SharedCacheClient.java      | 108 +++++
 .../client/api/impl/SharedCacheClientImpl.java  | 166 +++++++
 .../api/impl/TestSharedCacheClientImpl.java     | 170 +++++++
 .../src/main/resources/yarn-default.xml         |   2 +-
 .../client/impl/zk/RegistrySecurity.java        |  13 +-
 ...lineDelegationTokenSecretManagerService.java |  12 +-
 .../nodemanager/TestNodeManagerResync.java      |   1 +
 .../resourcemanager/RMActiveServiceContext.java |   4 +-
 .../server/resourcemanager/RMContextImpl.java   |  32 +-
 .../resourcemanager/RMSecretManagerService.java |  12 +-
 .../server/resourcemanager/rmapp/RMApp.java     |   3 +
 .../server/resourcemanager/rmapp/RMAppImpl.java |   5 +
 .../scheduler/capacity/CSQueueUtils.java        |  24 -
 .../scheduler/capacity/LeafQueue.java           | 252 ++++++----
 .../scheduler/common/fica/FiCaSchedulerApp.java |  15 +
 .../webapp/CapacitySchedulerPage.java           |   4 +-
 .../dao/CapacitySchedulerLeafQueueInfo.java     |  24 +-
 .../TestContainerResourceUsage.java             |   3 +-
 .../server/resourcemanager/TestRMRestart.java   |   3 +-
 .../applicationsmanager/MockAsm.java            |   7 +
 .../TestAMRMRPCNodeUpdates.java                 |   9 +
 .../applicationsmanager/TestAMRestart.java      |   6 +
 .../TestCapacitySchedulerPlanFollower.java      |   1 +
 .../server/resourcemanager/rmapp/MockRMApp.java |   7 +
 .../capacity/TestApplicationLimits.java         | 256 ++++++----
 .../scheduler/capacity/TestLeafQueue.java       |  79 ++--
 .../scheduler/capacity/TestReservations.java    |  34 +-
 .../scheduler/fifo/TestFifoScheduler.java       |  23 +-
 .../webapp/TestRMWebServicesCapacitySched.java  |  12 -
 .../src/site/apt/ResourceManagerHA.apt.vm       |   1 +
 .../src/site/apt/YarnCommands.apt.vm            |  52 +-
 83 files changed, 2782 insertions(+), 1316 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/630f1cd4/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------


[15/25] hadoop git commit: HADOOP-11318. Update the document for hadoop fs -stat

Posted by zj...@apache.org.
HADOOP-11318. Update the document for hadoop fs -stat


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

Branch: refs/heads/YARN-2928
Commit: ce29074685abaa88ae9586fe8277aca23f9d54d6
Parents: ce01176
Author: Akira Ajisaka <aa...@apache.org>
Authored: Fri Jan 16 00:55:20 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Fri Jan 16 00:55:20 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  2 ++
 .../java/org/apache/hadoop/fs/shell/Stat.java   | 33 +++++++++++++-------
 .../src/site/apt/FileSystemShell.apt.vm         | 11 +++++--
 .../src/test/resources/testConf.xml             | 24 +++++++++++---
 4 files changed, 51 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce290746/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 3f916a2..f80eeab 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -499,6 +499,8 @@ Release 2.7.0 - UNRELEASED
 
   BUG FIXES
 
+    HADOOP-11318. Update the document for hadoop fs -stat (aajisaka)
+
     HADOOP 11400. GraphiteSink does not reconnect to Graphite after 'broken pipe' 
     (Kamil Gorlo via raviprak)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce290746/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
index 6365294..458d3ee 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Stat.java
@@ -30,15 +30,17 @@ import org.apache.hadoop.fs.FileStatus;
 
 /**
  * Print statistics about path in specified format.
- * Format sequences:
- *   %b: Size of file in blocks
- *   %g: Group name of owner
- *   %n: Filename
- *   %o: Block size
- *   %r: replication
- *   %u: User name of owner
- *   %y: UTC date as &quot;yyyy-MM-dd HH:mm:ss&quot;
- *   %Y: Milliseconds since January 1, 1970 UTC
+ * Format sequences:<br>
+ *   %b: Size of file in blocks<br>
+ *   %F: Type<br>
+ *   %g: Group name of owner<br>
+ *   %n: Filename<br>
+ *   %o: Block size<br>
+ *   %r: replication<br>
+ *   %u: User name of owner<br>
+ *   %y: UTC date as &quot;yyyy-MM-dd HH:mm:ss&quot;<br>
+ *   %Y: Milliseconds since January 1, 1970 UTC<br>
+ * If the format is not specified, %y is used by default.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
@@ -48,12 +50,19 @@ class Stat extends FsCommand {
     factory.addClass(Stat.class, "-stat");
   }
 
+  private static final String NEWLINE = System.getProperty("line.separator");
+
   public static final String NAME = "stat";
   public static final String USAGE = "[format] <path> ...";
   public static final String DESCRIPTION =
-    "Print statistics about the file/directory at <path> " +
-    "in the specified format. Format accepts filesize in blocks (%b), group name of owner(%g), " +
-    "filename (%n), block size (%o), replication (%r), user name of owner(%u), modification date (%y, %Y)\n";
+    "Print statistics about the file/directory at <path>" + NEWLINE +
+    "in the specified format. Format accepts filesize in" + NEWLINE +
+    "blocks (%b), type (%F), group name of owner (%g)," + NEWLINE +
+    "name (%n), block size (%o), replication (%r), user name" + NEWLINE +
+    "of owner (%u), modification date (%y, %Y)." + NEWLINE +
+    "%y shows UTC date as \"yyyy-MM-dd HH:mm:ss\" and" + NEWLINE +
+    "%Y shows milliseconds since January 1, 1970 UTC." + NEWLINE +
+    "If the format is not specified, %y is used by default." + NEWLINE;
 
   protected final SimpleDateFormat timeFmt;
   {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce290746/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm b/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
index 757a0ba..5dfbd20 100644
--- a/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
+++ b/hadoop-common-project/hadoop-common/src/site/apt/FileSystemShell.apt.vm
@@ -665,13 +665,18 @@ permissions userid groupid modification_date modification_time dirname
 
 * stat
 
-   Usage: <<<hadoop fs -stat URI [URI ...]>>>
+   Usage: <<<hadoop fs -stat [format] \<path\> ...>>>
 
-   Returns the stat information on the path.
+   Print statistics about the file/directory at \<path\> in the specified
+   format. Format accepts filesize in blocks (%b), type (%F), group name of
+   owner (%g), name (%n), block size (%o), replication (%r), user name of
+   owner(%u), and modification date (%y, %Y). %y shows UTC date as
+   "yyyy-MM-dd HH:mm:ss" and %Y shows milliseconds since January 1, 1970 UTC.
+   If the format is not specified, %y is used by default.
 
    Example:
 
-     * <<<hadoop fs -stat path>>>
+     * <<<hadoop fs -stat "%F %u:%g %b %y %n" /file>>>
 
    Exit Code:
    Returns 0 on success and -1 on error.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce290746/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
index 5196641..5c667e1 100644
--- a/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
+++ b/hadoop-common-project/hadoop-common/src/test/resources/testConf.xml
@@ -786,19 +786,35 @@
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*Print statistics about the file/directory at &lt;path&gt; in the specified format.( )*</expected-output>
+          <expected-output>^( |\t)*Print statistics about the file/directory at &lt;path&gt;( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*Format accepts filesize in blocks \(%b\), group name of owner\(%g\), filename \(%n\),( )*</expected-output>
+          <expected-output>^( |\t)*in the specified format. Format accepts filesize in( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*block size \(%o\), replication \(%r\), user name of owner\(%u\), modification date( )*</expected-output>
+          <expected-output>^( |\t)*blocks \(%b\), type \(%F\), group name of owner \(%g\),( )*</expected-output>
         </comparator>
         <comparator>
           <type>RegexpComparator</type>
-          <expected-output>^( |\t)*\(%y, %Y\)( )*</expected-output>
+          <expected-output>^( |\t)*name \(%n\), block size \(%o\), replication \(%r\), user name( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*of owner \(%u\), modification date \(%y, %Y\).( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*%y shows UTC date as "yyyy-MM-dd HH:mm:ss" and( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*%Y shows milliseconds since January 1, 1970 UTC.( )*</expected-output>
+        </comparator>
+        <comparator>
+          <type>RegexpComparator</type>
+          <expected-output>^( |\t)*If the format is not specified, %y is used by default.( )*</expected-output>
         </comparator>
       </comparators>
     </test>


[09/25] hadoop git commit: HDFS-7067. ClassCastException while using a key created by keytool to create encryption zone. (Charles Lamb via Colin P. McCabe)

Posted by zj...@apache.org.
HDFS-7067. ClassCastException while using a key created by keytool to create encryption zone. (Charles Lamb via Colin P. McCabe)


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

Branch: refs/heads/YARN-2928
Commit: a5a033c7ca130528f8c509b035fc090ad4eb981a
Parents: 6464a89
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jan 14 17:42:59 2015 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Wed Jan 14 17:42:59 2015 -0800

----------------------------------------------------------------------
 .../hadoop/crypto/key/JavaKeyStoreProvider.java |   5 +-
 .../crypto/key/TestKeyProviderFactory.java      |  48 +++++++++++++++++++
 .../src/test/resources/hdfs7067.keystore        | Bin 0 -> 6067 bytes
 3 files changed, 52 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5a033c7/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 a9738ba..bfec1ef 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
@@ -21,7 +21,6 @@ package org.apache.hadoop.crypto.key;
 import com.google.common.base.Preconditions;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -401,6 +400,10 @@ public class JavaKeyStoreProvider extends KeyProvider {
         Metadata meta = ((KeyMetadata) keyStore.getKey(name, password)).metadata;
         cache.put(name, meta);
         return meta;
+      } catch (ClassCastException e) {
+        throw new IOException("Can't cast key for " + name + " in keystore " +
+            path + " to a KeyMetadata. Key may have been added using " +
+            " keytool or some other non-Hadoop method.", e);
       } catch (KeyStoreException e) {
         throw new IOException("Can't get metadata for " + name +
             " from keystore " + path, e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5a033c7/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
index 998cd6f..ef09d94 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestKeyProviderFactory.java
@@ -42,6 +42,7 @@ import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
 
 public class TestKeyProviderFactory {
 
@@ -430,4 +431,51 @@ public class TestKeyProviderFactory {
     Assert.assertNull(kp);
 
   }
+
+  @Test
+  public void testJksProviderWithKeytoolKeys() throws Exception {
+    final Configuration conf = new Configuration();
+    final String keystoreDirAbsolutePath =
+        conf.getResource("hdfs7067.keystore").getPath();
+    final String ourUrl = JavaKeyStoreProvider.SCHEME_NAME + "://file@/" +
+        keystoreDirAbsolutePath;
+
+    conf.set(KeyProviderFactory.KEY_PROVIDER_PATH, ourUrl);
+
+    final KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
+
+    // Sanity check that we are using the right keystore
+    @SuppressWarnings("unused")
+    final KeyProvider.KeyVersion keyVersion =
+            provider.getKeyVersion("testkey5@0");
+    try {
+      @SuppressWarnings("unused")
+      final KeyProvider.KeyVersion keyVersionWrongKeyNameFormat =
+          provider.getKeyVersion("testkey2");
+      fail("should have thrown an exception");
+    } catch (IOException e) {
+      // No version in key path testkey2/
+      GenericTestUtils.assertExceptionContains("No version in key path", e);
+    }
+    try {
+      @SuppressWarnings("unused")
+      final KeyProvider.KeyVersion keyVersionCurrentKeyNotWrongKeyNameFormat =
+          provider.getCurrentKey("testkey5@0");
+      fail("should have thrown an exception getting testkey5@0");
+    } catch (IOException e) {
+      // javax.crypto.spec.SecretKeySpec cannot be cast to
+      // org.apache.hadoop.crypto.key.JavaKeyStoreProvider$KeyMetadata
+      GenericTestUtils.assertExceptionContains("other non-Hadoop method", e);
+    }
+    try {
+      @SuppressWarnings("unused")
+      KeyProvider.KeyVersion keyVersionCurrentKeyNotReally =
+          provider.getCurrentKey("testkey2");
+      fail("should have thrown an exception getting testkey2");
+    } catch (IOException e) {
+      // javax.crypto.spec.SecretKeySpec cannot be cast to
+      // org.apache.hadoop.crypto.key.JavaKeyStoreProvider$KeyMetadata
+      GenericTestUtils.assertExceptionContains("other non-Hadoop method", e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a5a033c7/hadoop-common-project/hadoop-common/src/test/resources/hdfs7067.keystore
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/resources/hdfs7067.keystore b/hadoop-common-project/hadoop-common/src/test/resources/hdfs7067.keystore
new file mode 100644
index 0000000..a0a69b1
Binary files /dev/null and b/hadoop-common-project/hadoop-common/src/test/resources/hdfs7067.keystore differ


[17/25] hadoop git commit: HDFS-7591. hdfs classpath command should support same options as hadoop classpath. (Contributed by Varun Saxena)

Posted by zj...@apache.org.
HDFS-7591. hdfs classpath command should support same options as hadoop classpath. (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/a6efbb21
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a6efbb21
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a6efbb21

Branch: refs/heads/YARN-2928
Commit: a6efbb21c2561f718f02a788d5fd25c02eff09a0
Parents: ce29074
Author: Arpit Agarwal <ar...@apache.org>
Authored: Thu Jan 15 11:02:17 2015 -0800
Committer: Arpit Agarwal <ar...@apache.org>
Committed: Thu Jan 15 11:02:17 2015 -0800

----------------------------------------------------------------------
 .../hadoop-common/src/main/bin/hadoop                 |  8 +-------
 .../hadoop-common/src/main/bin/hadoop-functions.sh    | 11 +++++++++++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt           |  3 +++
 hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs     |  4 +---
 hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd | 14 +++++++++++++-
 5 files changed, 29 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6efbb21/hadoop-common-project/hadoop-common/src/main/bin/hadoop
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop b/hadoop-common-project/hadoop-common/src/main/bin/hadoop
index b216b8f..6ba327a 100755
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop
@@ -124,13 +124,7 @@ case ${COMMAND} in
     CLASS=org.apache.hadoop.util.NativeLibraryChecker
   ;;
   classpath)
-    if [[ "$#" -eq 1 ]]; then
-      CLASS=org.apache.hadoop.util.Classpath
-    else   
-      hadoop_finalize
-      echo "${CLASSPATH}"
-      exit 0
-    fi
+    hadoop_do_classpath_subcommand "$@"
   ;;
   credential)
     CLASS=org.apache.hadoop.security.alias.CredentialShell

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6efbb21/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
index 75d2499..f70be5a 100644
--- a/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
+++ b/hadoop-common-project/hadoop-common/src/main/bin/hadoop-functions.sh
@@ -1240,3 +1240,14 @@ function hadoop_verify_user
     fi
   fi
 }
+
+function hadoop_do_classpath_subcommand
+{
+  if [[ "$#" -gt 0 ]]; then
+    CLASS=org.apache.hadoop.util.Classpath
+  else
+    hadoop_finalize
+    echo "${CLASSPATH}"
+    exit 0
+  fi
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6efbb21/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 63797eb..c3d32b8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -133,6 +133,9 @@ Trunk (Unreleased)
     type instead of concrete classes Block and ReplicaInfo.  (David Powell
     and Joe Pallas via szetszwo)
 
+    HDFS-7591. hdfs classpath command should support same options as hadoop
+    classpath (Varun Saxena via Arpit Agarwal)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6efbb21/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
index 650a0f5..5f8ff18 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs
@@ -94,9 +94,7 @@ case ${COMMAND} in
     CLASS=org.apache.hadoop.hdfs.tools.CacheAdmin
   ;;
   classpath)
-    hadoop_finalize
-    echo "${CLASSPATH}"
-    exit
+    hadoop_do_classpath_subcommand "$@"
   ;;
   crypto)
     CLASS=org.apache.hadoop.hdfs.tools.CryptoAdmin

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6efbb21/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
index 851637d..7883252 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs.cmd
@@ -51,7 +51,15 @@ if "%1" == "--loglevel" (
       goto print_usage
   )
 
-  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies
+  if %hdfs-command% == classpath (
+    if not defined hdfs-command-arguments (
+      @rem No need to bother starting up a JVM for this simple case.
+      @echo %CLASSPATH%
+      exit /b
+    )
+  )
+
+  set hdfscommands=dfs namenode secondarynamenode journalnode zkfc datanode dfsadmin haadmin fsck balancer jmxget oiv oev fetchdt getconf groups snapshotDiff lsSnapshottableDir cacheadmin mover storagepolicies classpath
   for %%i in ( %hdfscommands% ) do (
     if %hdfs-command% == %%i set hdfscommand=true
   )
@@ -122,6 +130,10 @@ goto :eof
   set CLASS=org.apache.hadoop.hdfs.tools.JMXGet
   goto :eof
 
+:classpath
+  set CLASS=org.apache.hadoop.util.Classpath
+  goto :eof
+
 :oiv
   set CLASS=org.apache.hadoop.hdfs.tools.offlineImageViewer.OfflineImageViewerPB
   goto :eof


[23/25] hadoop git commit: YARN-3064. TestRMRestart/TestContainerResourceUsage/TestNodeManagerResync failure with allocation timeout. (Contributed by Jian He)

Posted by zj...@apache.org.
YARN-3064. TestRMRestart/TestContainerResourceUsage/TestNodeManagerResync failure with allocation timeout. (Contributed by Jian He)


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

Branch: refs/heads/YARN-2928
Commit: 5d1cca34fa14706586f8bbe7f2a2264757fdcc11
Parents: c4ccbe6
Author: Junping Du <ju...@apache.org>
Authored: Fri Jan 16 00:08:36 2015 -0800
Committer: Junping Du <ju...@apache.org>
Committed: Fri Jan 16 00:08:36 2015 -0800

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                                | 3 +++
 .../hadoop/yarn/server/nodemanager/TestNodeManagerResync.java  | 1 +
 .../server/resourcemanager/TestContainerResourceUsage.java     | 3 ++-
 .../hadoop/yarn/server/resourcemanager/TestRMRestart.java      | 3 ++-
 .../resourcemanager/applicationsmanager/TestAMRestart.java     | 6 ++++++
 5 files changed, 14 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d1cca34/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 9173deb..d069f6d 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -374,6 +374,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2861. Fixed Timeline DT secret manager to not reuse RM's configs.
     (Zhijie Shen via jianhe)
 
+    YARN-3064. TestRMRestart/TestContainerResourceUsage/TestNodeManagerResync 
+    failure with allocation timeout. (Jian He via junping_du)
+
 Release 2.6.0 - 2014-11-18
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d1cca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.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/TestNodeManagerResync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
index a58294f..611e671 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
@@ -165,6 +165,7 @@ public class TestNodeManagerResync {
       throws IOException, InterruptedException, YarnException {
     NodeManager nm = new TestNodeManager2();
     YarnConfiguration conf = createNMConfig();
+    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
     nm.init(conf);
     nm.start();
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d1cca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.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/TestContainerResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
index b9397bf..fcb48a0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
@@ -61,7 +61,6 @@ public class TestContainerResourceUsage {
     rootLogger.setLevel(Level.DEBUG);
     conf = new YarnConfiguration();
     UserGroupInformation.setConfiguration(conf);
-    conf.set(YarnConfiguration.RECOVERY_ENABLED, "true");
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
         YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
   }
@@ -137,6 +136,8 @@ public class TestContainerResourceUsage {
     // Set max attempts to 1 so that when the first attempt fails, the app
     // won't try to start a new one.
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
+    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
     MemoryRMStateStore memStore = new MemoryRMStateStore();
     memStore.init(conf);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d1cca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.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/TestRMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
index b3202f2..bd6fdba 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
@@ -134,7 +134,8 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
     Logger rootLogger = LogManager.getRootLogger();
     rootLogger.setLevel(Level.DEBUG);
     UserGroupInformation.setConfiguration(conf);
-    conf.set(YarnConfiguration.RECOVERY_ENABLED, "true");
+    conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     rmAddr = new InetSocketAddress("localhost", 8032);
     Assert.assertTrue(YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS > 1);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5d1cca34/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.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/applicationsmanager/TestAMRestart.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
index 49b1841..7befba4 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
@@ -470,6 +470,8 @@ public class TestAMRestart {
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
       ResourceScheduler.class);
     conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
+
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     // explicitly set max-am-retry count as 1.
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
@@ -535,6 +537,8 @@ public class TestAMRestart {
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
       ResourceScheduler.class);
     conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
+
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     // explicitly set max-am-retry count as 1.
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 1);
@@ -594,6 +598,8 @@ public class TestAMRestart {
     conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class,
       ResourceScheduler.class);
     conf.setBoolean(YarnConfiguration.RECOVERY_ENABLED, true);
+    conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
+
     conf.set(YarnConfiguration.RM_STORE, MemoryRMStateStore.class.getName());
     // explicitly set max-am-retry count as 2.
     conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 2);


[07/25] hadoop git commit: HDFS-2219. Change fsck to support fully qualified paths so that a particular namenode in a federated cluster with multiple namenodes can be specified in the path parameter.

Posted by zj...@apache.org.
HDFS-2219. Change fsck to support fully qualified paths so that a particular namenode in a federated cluster with multiple namenodes can be specified in the path parameter.


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

Branch: refs/heads/YARN-2928
Commit: 7fe0f25ad21f006eb41b832a181eb2a812a6f7b7
Parents: 446545c
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Wed Jan 14 11:31:18 2015 -0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Wed Jan 14 11:31:18 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   4 +
 .../org/apache/hadoop/hdfs/tools/DFSck.java     |  32 +++--
 .../namenode/TestFsckWithMultipleNameNodes.java | 141 +++++++++++++++++++
 3 files changed, 163 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe0f25a/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 421d2f0..b879f62d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -502,6 +502,10 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7600. Refine hdfs admin classes to reuse common code. (jing9)
 
+    HDFS-2219. Change fsck to support fully qualified paths so that a 
+    particular namenode in a federated cluster with multiple namenodes
+    can be specified in the path parameter.  (szetszwo)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe0f25a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
index 0d73b43..98f2030 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -229,14 +230,14 @@ public class DFSck extends Configured implements Tool {
    * @return Returns http address or null if failure.
    * @throws IOException if we can't determine the active NN address
    */
-  private URI getCurrentNamenodeAddress() throws IOException {
+  private URI getCurrentNamenodeAddress(Path target) throws IOException {
     //String nnAddress = null;
     Configuration conf = getConf();
 
     //get the filesystem object to verify it is an HDFS system
-    FileSystem fs;
+    final FileSystem fs;
     try {
-      fs = FileSystem.get(conf);
+      fs = target.getFileSystem(conf);
     } catch (IOException ioe) {
       System.err.println("FileSystem is inaccessible due to:\n"
           + StringUtils.stringifyException(ioe));
@@ -254,16 +255,6 @@ public class DFSck extends Configured implements Tool {
   private int doWork(final String[] args) throws IOException {
     final StringBuilder url = new StringBuilder();
     
-    URI namenodeAddress = getCurrentNamenodeAddress();
-    if (namenodeAddress == null) {
-      //Error message already output in {@link #getCurrentNamenodeAddress()}
-      System.err.println("DFSck exiting.");
-      return 0;
-    }
-
-    url.append(namenodeAddress.toString());
-    System.err.println("Connecting to namenode via " + url.toString());
-    
     url.append("/fsck?ugi=").append(ugi.getShortUserName());
     String dir = null;
     boolean doListCorruptFileBlocks = false;
@@ -309,7 +300,20 @@ public class DFSck extends Configured implements Tool {
     if (null == dir) {
       dir = "/";
     }
-    url.append("&path=").append(URLEncoder.encode(dir, "UTF-8"));
+
+    final Path dirpath = new Path(dir);
+    final URI namenodeAddress = getCurrentNamenodeAddress(dirpath);
+    if (namenodeAddress == null) {
+      //Error message already output in {@link #getCurrentNamenodeAddress()}
+      System.err.println("DFSck exiting.");
+      return 0;
+    }
+
+    url.insert(0, namenodeAddress.toString());
+    url.append("&path=").append(URLEncoder.encode(
+        Path.getPathWithoutSchemeAndAuthority(dirpath).toString(), "UTF-8"));
+    System.err.println("Connecting to namenode via " + url.toString());
+
     if (doListCorruptFileBlocks) {
       return listCorruptFileBlocks(dir, url.toString());
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/7fe0f25a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java
new file mode 100644
index 0000000..518a003
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java
@@ -0,0 +1,141 @@
+/**
+ * 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.util.Random;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.commons.logging.impl.Log4JLogger;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+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.MiniDFSNNTopology;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.server.balancer.TestBalancer;
+import org.apache.log4j.Level;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test fsck with multiple NameNodes
+ */
+public class TestFsckWithMultipleNameNodes {
+  static final Log LOG = LogFactory.getLog(TestFsckWithMultipleNameNodes.class);
+  {
+    ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.OFF);
+    ((Log4JLogger)LeaseManager.LOG).getLogger().setLevel(Level.OFF);
+    ((Log4JLogger)LogFactory.getLog(FSNamesystem.class)).getLogger().setLevel(Level.OFF);
+  }
+
+  
+  private static final String FILE_NAME = "/tmp.txt";
+  private static final Path FILE_PATH = new Path(FILE_NAME);
+  
+  private static final Random RANDOM = new Random();
+
+  static {
+    TestBalancer.initTestSetup();
+  }
+
+  /** Common objects used in various methods. */
+  private static class Suite {
+    final MiniDFSCluster cluster;
+    final ClientProtocol[] clients;
+    final short replication;
+    
+    Suite(MiniDFSCluster cluster, final int nNameNodes, final int nDataNodes)
+        throws IOException {
+      this.cluster = cluster;
+      clients = new ClientProtocol[nNameNodes];
+      for(int i = 0; i < nNameNodes; i++) {
+        clients[i] = cluster.getNameNode(i).getRpcServer();
+      }
+      replication = (short)Math.max(1, nDataNodes - 1);
+    }
+
+    /** create a file with a length of <code>fileLen</code> */
+    private void createFile(int index, long len
+        ) throws IOException, InterruptedException, TimeoutException {
+      final FileSystem fs = cluster.getFileSystem(index);
+      DFSTestUtil.createFile(fs, FILE_PATH, len, replication, RANDOM.nextLong());
+      DFSTestUtil.waitReplication(fs, FILE_PATH, replication);
+    }
+
+  }
+
+  private static Configuration createConf() {
+    final Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 1L);
+    conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L);
+    return conf;
+  }
+
+  private void runTest(final int nNameNodes, final int nDataNodes,
+      Configuration conf) throws Exception {
+    LOG.info("nNameNodes=" + nNameNodes + ", nDataNodes=" + nDataNodes);
+
+    LOG.info("RUN_TEST -1");
+    final MiniDFSCluster cluster = new MiniDFSCluster
+        .Builder(conf)
+        .nnTopology(MiniDFSNNTopology.simpleFederatedTopology(nNameNodes))
+        .numDataNodes(nDataNodes)
+        .build();
+    LOG.info("RUN_TEST 0");
+    DFSTestUtil.setFederatedConfiguration(cluster, conf);
+
+    try {
+      cluster.waitActive();
+      LOG.info("RUN_TEST 1");
+      final Suite s = new Suite(cluster, nNameNodes, nDataNodes);
+      for(int i = 0; i < nNameNodes; i++) {
+        s.createFile(i, 1024);
+      }
+
+      LOG.info("RUN_TEST 2");
+      final String[] urls = new String[nNameNodes];
+      for(int i = 0; i < urls.length; i++) {
+        urls[i] = cluster.getFileSystem(i).getUri() + FILE_NAME;
+        LOG.info("urls[" + i + "]=" + urls[i]);
+        final String result = TestFsck.runFsck(conf, 0, false, urls[i]);
+        LOG.info("result=" + result);
+        Assert.assertTrue(result.contains("Status: HEALTHY"));
+      }
+    } finally {
+      cluster.shutdown();
+    }
+    LOG.info("RUN_TEST 6");
+  }
+  
+  /** Test a cluster with even distribution, 
+   * then a new empty node is added to the cluster
+   */
+  @Test
+  public void testFsck() throws Exception {
+    final Configuration conf = createConf();
+    runTest(3, 1, conf);
+  }
+
+}
+


[14/25] hadoop git commit: HDFS-7581. HDFS documentation needs updating post-shell rewrite (aw)

Posted by zj...@apache.org.
HDFS-7581. HDFS documentation needs updating post-shell rewrite (aw)


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

Branch: refs/heads/YARN-2928
Commit: ce0117636af8753ea326fd76b08478addebc10a6
Parents: 533e551
Author: Allen Wittenauer <aw...@apache.org>
Authored: Thu Jan 15 07:48:55 2015 -0800
Committer: Allen Wittenauer <aw...@apache.org>
Committed: Thu Jan 15 07:48:55 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   2 +
 .../hadoop-hdfs/src/site/apt/Federation.apt.vm  | 158 +++----
 .../src/site/apt/HDFSCommands.apt.vm            | 471 +++++++++++++++----
 .../site/apt/HDFSHighAvailabilityWithNFS.apt.vm |  70 +--
 .../site/apt/HDFSHighAvailabilityWithQJM.apt.vm |  40 +-
 .../src/site/apt/HdfsNfsGateway.apt.vm          | 108 ++---
 6 files changed, 561 insertions(+), 288 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce011763/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 11d15f8..63797eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -265,6 +265,8 @@ Trunk (Unreleased)
 
     HDFS-7407. Minor typo in privileged pid/out/log names (aw)
 
+    HDFS-7581. HDFS documentation needs updating post-shell rewrite (aw)
+
 Release 2.7.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce011763/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm
index 3d27b8e..29278b7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Federation.apt.vm
@@ -34,40 +34,40 @@ HDFS Federation
 
     * Consists of directories, files and blocks
 
-    * It supports all the namespace related file system operations such as 
+    * It supports all the namespace related file system operations such as
       create, delete, modify and list files and directories.
 
   * <<Block Storage Service>> has two parts
 
     * Block Management (which is done in Namenode)
 
-      * Provides datanode cluster membership by handling registrations, and 
+      * Provides datanode cluster membership by handling registrations, and
         periodic heart beats.
 
       * Processes block reports and maintains location of blocks.
 
-      * Supports block related operations such as create, delete, modify and 
+      * Supports block related operations such as create, delete, modify and
         get block location.
 
-      * Manages replica placement and replication of a block for under 
+      * Manages replica placement and replication of a block for under
         replicated blocks and deletes blocks that are over replicated.
 
-    * Storage - is provided by datanodes by storing blocks on the local file 
+    * Storage - is provided by datanodes by storing blocks on the local file
       system and allows read/write access.
 
-  The prior HDFS architecture allows only a single namespace for the 
-  entire cluster. A single Namenode manages this namespace. HDFS 
-  Federation addresses limitation of the prior architecture by adding 
+  The prior HDFS architecture allows only a single namespace for the
+  entire cluster. A single Namenode manages this namespace. HDFS
+  Federation addresses limitation of the prior architecture by adding
   support multiple Namenodes/namespaces to HDFS file system.
-    
+
 * {Multiple Namenodes/Namespaces}
 
-  In order to scale the name service horizontally, federation uses multiple 
-  independent Namenodes/namespaces. The Namenodes are federated, that is, the 
-  Namenodes are independent and don’t require coordination with each other. 
-  The datanodes are used as common storage for blocks by all the Namenodes. 
-  Each datanode registers with all the Namenodes in the cluster. Datanodes 
-  send periodic heartbeats and block reports and handles commands from the 
+  In order to scale the name service horizontally, federation uses multiple
+  independent Namenodes/namespaces. The Namenodes are federated, that is, the
+  Namenodes are independent and do not require coordination with each other.
+  The datanodes are used as common storage for blocks by all the Namenodes.
+  Each datanode registers with all the Namenodes in the cluster. Datanodes
+  send periodic heartbeats and block reports and handles commands from the
   Namenodes.
 
   Users may use {{{./ViewFs.html}ViewFs}} to create personalized namespace views,
@@ -78,48 +78,48 @@ HDFS Federation
 
   <<Block Pool>>
 
-  A Block Pool is a set of blocks that belong to a single namespace. 
+  A Block Pool is a set of blocks that belong to a single namespace.
   Datanodes store blocks for all the block pools in the cluster.
-  It is managed independently of other block pools. This allows a namespace 
-  to generate Block IDs for new blocks without the need for coordination 
-  with the other namespaces. The failure of a Namenode does not prevent 
+  It is managed independently of other block pools. This allows a namespace
+  to generate Block IDs for new blocks without the need for coordination
+  with the other namespaces. The failure of a Namenode does not prevent
   the datanode from serving other Namenodes in the cluster.
 
-  A Namespace and its block pool together are called Namespace Volume. 
-  It is a self-contained unit of management. When a Namenode/namespace 
+  A Namespace and its block pool together are called Namespace Volume.
+  It is a self-contained unit of management. When a Namenode/namespace
   is deleted, the corresponding block pool at the datanodes is deleted.
   Each namespace volume is upgraded as a unit, during cluster upgrade.
 
   <<ClusterID>>
 
-  A new identifier <<ClusterID>> is added to identify all the nodes in 
-  the cluster.  When a Namenode is formatted, this identifier is provided 
-  or auto generated. This ID should be used for formatting the other 
+  A new identifier <<ClusterID>> is added to identify all the nodes in
+  the cluster.  When a Namenode is formatted, this identifier is provided
+  or auto generated. This ID should be used for formatting the other
   Namenodes into the cluster.
 
 ** Key Benefits
 
-  * Namespace Scalability - HDFS cluster storage scales horizontally but 
-    the namespace does not. Large deployments or deployments using lot 
-    of small files benefit from scaling the namespace by adding more 
+  * Namespace Scalability - HDFS cluster storage scales horizontally but
+    the namespace does not. Large deployments or deployments using lot
+    of small files benefit from scaling the namespace by adding more
     Namenodes to the cluster
 
   * Performance - File system operation throughput is limited by a single
     Namenode in the prior architecture. Adding more Namenodes to the cluster
     scales the file system read/write operations throughput.
 
-  * Isolation - A single Namenode offers no isolation in multi user 
-    environment. An experimental application can overload the Namenode 
-    and slow down production critical applications. With multiple Namenodes, 
-    different categories of applications and users can be isolated to 
+  * Isolation - A single Namenode offers no isolation in multi user
+    environment. An experimental application can overload the Namenode
+    and slow down production critical applications. With multiple Namenodes,
+    different categories of applications and users can be isolated to
     different namespaces.
 
 * {Federation Configuration}
 
-  Federation configuration is <<backward compatible>> and allows existing 
-  single Namenode configuration to work without any change. The new 
-  configuration is designed such that all the nodes in the cluster have 
-  same configuration without the need for deploying different configuration 
+  Federation configuration is <<backward compatible>> and allows existing
+  single Namenode configuration to work without any change. The new
+  configuration is designed such that all the nodes in the cluster have
+  same configuration without the need for deploying different configuration
   based on the type of the node in the cluster.
 
   A new abstraction called <<<NameServiceID>>> is added with
@@ -132,12 +132,12 @@ HDFS Federation
 ** Configuration:
 
   <<Step 1>>: Add the following parameters to your configuration:
-  <<<dfs.nameservices>>>: Configure with list of comma separated 
-  NameServiceIDs. This will be used by Datanodes to determine all the 
+  <<<dfs.nameservices>>>: Configure with list of comma separated
+  NameServiceIDs. This will be used by Datanodes to determine all the
   Namenodes in the cluster.
-  
-  <<Step 2>>: For each Namenode and Secondary Namenode/BackupNode/Checkpointer 
-  add the following configuration suffixed with the corresponding 
+
+  <<Step 2>>: For each Namenode and Secondary Namenode/BackupNode/Checkpointer
+  add the following configuration suffixed with the corresponding
   <<<NameServiceID>>> into the common configuration file.
 
 *---------------------+--------------------------------------------+
@@ -159,7 +159,7 @@ HDFS Federation
 | BackupNode          | <<<dfs.namenode.backup.address>>>          |
 |                     | <<<dfs.secondary.namenode.keytab.file>>>   |
 *---------------------+--------------------------------------------+
-    
+
   Here is an example configuration with two namenodes:
 
 ----
@@ -200,31 +200,31 @@ HDFS Federation
 ** Formatting Namenodes
 
   <<Step 1>>: Format a namenode using the following command:
-  
+
 ----
-> $HADOOP_PREFIX_HOME/bin/hdfs namenode -format [-clusterId <cluster_id>]
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs namenode -format [-clusterId <cluster_id>]
 ----
-  Choose a unique cluster_id, which will not conflict other clusters in 
-  your environment. If it is not provided, then a unique ClusterID is 
+  Choose a unique cluster_id, which will not conflict other clusters in
+  your environment. If it is not provided, then a unique ClusterID is
   auto generated.
 
   <<Step 2>>: Format additional namenode using the following command:
 
 ----
-> $HADOOP_PREFIX_HOME/bin/hdfs namenode -format -clusterId <cluster_id>
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs namenode -format -clusterId <cluster_id>
 ----
-  Note that the cluster_id in step 2 must be same as that of the 
-  cluster_id in step 1. If they are different, the additional Namenodes 
+  Note that the cluster_id in step 2 must be same as that of the
+  cluster_id in step 1. If they are different, the additional Namenodes
   will not be part of the federated cluster.
 
 ** Upgrading from an older release and configuring federation
 
-  Older releases supported a single Namenode. 
+  Older releases supported a single Namenode.
   Upgrade the cluster to newer release to enable federation
   During upgrade you can provide a ClusterID as follows:
 
 ----
-> $HADOOP_PREFIX_HOME/bin/hdfs start namenode --config $HADOOP_CONF_DIR  -upgrade -clusterId <cluster_ID>
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs start namenode --config $HADOOP_CONF_DIR  -upgrade -clusterId <cluster_ID>
 ----
   If ClusterID is not provided, it is auto generated.
 
@@ -234,8 +234,8 @@ HDFS Federation
 
   * Add configuration parameter <<<dfs.nameservices>>> to the configuration.
 
-  * Update the configuration with NameServiceID suffix. Configuration 
-    key names have changed post release 0.20. You must use new configuration 
+  * Update the configuration with NameServiceID suffix. Configuration
+    key names have changed post release 0.20. You must use new configuration
     parameter names, for federation.
 
   * Add new Namenode related config to the configuration files.
@@ -244,11 +244,11 @@ HDFS Federation
 
   * Start the new Namenode, Secondary/Backup.
 
-  * Refresh the datanodes to pickup the newly added Namenode by running 
+  * Refresh the datanodes to pickup the newly added Namenode by running
     the following command:
 
 ----
-> $HADOOP_PREFIX_HOME/bin/hdfs dfadmin -refreshNameNode <datanode_host_name>:<datanode_rpc_port>
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs dfadmin -refreshNameNode <datanode_host_name>:<datanode_rpc_port>
 ----
 
   * The above command must be run against all the datanodes in the cluster.
@@ -260,37 +260,37 @@ HDFS Federation
   To start the cluster run the following command:
 
 ----
-> $HADOOP_PREFIX_HOME/bin/start-dfs.sh
+[hdfs]$ $HADOOP_PREFIX/sbin/start-dfs.sh
 ----
 
   To stop the cluster run the following command:
 
 ----
-> $HADOOP_PREFIX_HOME/bin/stop-dfs.sh
+[hdfs]$ $HADOOP_PREFIX/sbin/stop-dfs.sh
 ----
 
-  These commands can be run from any node where the HDFS configuration is 
-  available.  The command uses configuration to determine the Namenodes 
-  in the cluster and starts the Namenode process on those nodes. The 
-  datanodes are started on nodes specified in the <<<slaves>>> file. The 
-  script can be used as reference for building your own scripts for 
+  These commands can be run from any node where the HDFS configuration is
+  available.  The command uses configuration to determine the Namenodes
+  in the cluster and starts the Namenode process on those nodes. The
+  datanodes are started on nodes specified in the <<<slaves>>> file. The
+  script can be used as reference for building your own scripts for
   starting and stopping the cluster.
 
 **  Balancer
 
-  Balancer has been changed to work with multiple Namenodes in the cluster to 
+  Balancer has been changed to work with multiple Namenodes in the cluster to
   balance the cluster. Balancer can be run using the command:
 
 ----
-"$HADOOP_PREFIX"/bin/hadoop-daemon.sh --config $HADOOP_CONF_DIR --script "$bin"/hdfs start balancer [-policy <policy>]
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon start balancer [-policy <policy>]
 ----
 
   Policy could be:
 
-  * <<<datanode>>> - this is the <default> policy. This balances the storage at 
+  * <<<datanode>>> - this is the <default> policy. This balances the storage at
     the datanode level. This is similar to balancing policy from prior releases.
 
-  * <<<blockpool>>> - this balances the storage at the block pool level. 
+  * <<<blockpool>>> - this balances the storage at the block pool level.
     Balancing at block pool level balances storage at the datanode level also.
 
   Note that Balancer only balances the data and does not balance the namespace.
@@ -298,43 +298,43 @@ HDFS Federation
 
 ** Decommissioning
 
-  Decommissioning is similar to prior releases. The nodes that need to be 
-  decomissioned are added to the exclude file at all the Namenode. Each 
-  Namenode decommissions its Block Pool. When all the Namenodes finish 
+  Decommissioning is similar to prior releases. The nodes that need to be
+  decomissioned are added to the exclude file at all the Namenode. Each
+  Namenode decommissions its Block Pool. When all the Namenodes finish
   decommissioning a datanode, the datanode is considered to be decommissioned.
 
-  <<Step 1>>: To distributed an exclude file to all the Namenodes, use the 
+  <<Step 1>>: To distributed an exclude file to all the Namenodes, use the
   following command:
 
 ----
-"$HADOOP_PREFIX"/bin/distributed-exclude.sh <exclude_file>
+[hdfs]$ $HADOOP_PREFIX/sbin/distributed-exclude.sh <exclude_file>
 ----
 
   <<Step 2>>: Refresh all the Namenodes to pick up the new exclude file.
 
 ----
-"$HADOOP_PREFIX"/bin/refresh-namenodes.sh
+[hdfs]$ $HADOOP_PREFIX/sbin/refresh-namenodes.sh
 ----
- 
-  The above command uses HDFS configuration to determine the Namenodes 
-  configured in the cluster and refreshes all the Namenodes to pick up 
+
+  The above command uses HDFS configuration to determine the Namenodes
+  configured in the cluster and refreshes all the Namenodes to pick up
   the new exclude file.
 
 ** Cluster Web Console
 
-  Similar to Namenode status web page, a Cluster Web Console is added in 
-  federation to monitor the federated cluster at 
+  Similar to Namenode status web page, a Cluster Web Console is added in
+  federation to monitor the federated cluster at
   <<<http://<any_nn_host:port>/dfsclusterhealth.jsp>>>.
   Any Namenode in the cluster can be used to access this web page.
 
   The web page provides the following information:
 
-  * Cluster summary that shows number of files, number of blocks and 
-    total configured storage capacity, available and used storage information 
+  * Cluster summary that shows number of files, number of blocks and
+    total configured storage capacity, available and used storage information
     for the entire cluster.
 
   * Provides list of Namenodes and summary that includes number of files,
-    blocks, missing blocks, number of live and dead data nodes for each 
+    blocks, missing blocks, number of live and dead data nodes for each
     Namenode. It also provides a link to conveniently access Namenode web UI.
 
   * It also provides decommissioning status of datanodes.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce011763/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSCommands.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSCommands.apt.vm b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSCommands.apt.vm
index 4cf31ac..941a8ee 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSCommands.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSCommands.apt.vm
@@ -18,7 +18,7 @@
 
 HDFS Commands Guide
 
-%{toc|section=1|fromDepth=2|toDepth=4}
+%{toc|section=1|fromDepth=2|toDepth=3}
 
 * Overview
 
@@ -26,39 +26,37 @@ HDFS Commands Guide
    hdfs script without any arguments prints the description for all
    commands.
 
-   Usage: <<<hdfs [--config confdir] [--loglevel loglevel] [COMMAND]
-          [GENERIC_OPTIONS] [COMMAND_OPTIONS]>>>
-
-   Hadoop has an option parsing framework that employs parsing generic options
-   as well as running classes.
-
-*-----------------------+---------------+
-|| COMMAND_OPTION       || Description
-*-----------------------+---------------+
-| <<<--config confdir>>>| Overwrites the default Configuration directory.
-|                       | Default is <<<${HADOOP_HOME}/conf>>>.
-*-----------------------+---------------+
-| <<<--loglevel loglevel>>>| Overwrites the log level. Valid log levels are
-|                       | FATAL, ERROR, WARN, INFO, DEBUG, and TRACE.
-|                       | Default is INFO.
-*-----------------------+---------------+
-| GENERIC_OPTIONS       | The common set of options supported by multiple
-|                       | commands. Full list is
-|                       | {{{../hadoop-common/CommandsManual.html#Generic_Options}here}}.
-*-----------------------+---------------+
-| COMMAND_OPTIONS       | Various commands with their options are described in
-|                       | the following sections. The commands have been
-|                       | grouped into {{{User Commands}}} and
-|                       | {{{Administration Commands}}}.
-*-----------------------+---------------+
-
-* User Commands
+ Usage: <<<hdfs [SHELL_OPTIONS] COMMAND [GENERIC_OPTIONS] [COMMAND_OPTIONS]>>>
+
+  Hadoop has an option parsing framework that employs parsing generic options as
+  well as running classes.
+
+*---------------+--------------+
+|| COMMAND_OPTIONS || Description                   |
+*-------------------------+-------------+
+| SHELL_OPTIONS | The common set of shell options. These are documented on the {{{../../hadoop-project-dist/hadoop-common/CommandsManual.html#Shell Options}Commands Manual}} page.
+*-------------------------+----+
+| GENERIC_OPTIONS | The common set of options supported by multiple commands. See the Hadoop {{{../../hadoop-project-dist/hadoop-common/CommandsManual.html#Generic Options}Commands Manual}} for more information.
+*------------------+---------------+
+| COMMAND COMMAND_OPTIONS | Various commands with their options are described
+|                         | in the following sections. The commands have been
+|                         | grouped into {{User Commands}} and
+|                         | {{Administration Commands}}.
+*-------------------------+--------------+
+
+* {User Commands}
 
    Commands useful for users of a hadoop cluster.
 
+** <<<classpath>>>
+
+  Usage: <<<hdfs classpath>>>
+
+  Prints the class path needed to get the Hadoop jar and the required libraries
+
 ** <<<dfs>>>
 
-   Usage: <<<hdfs dfs [GENERIC_OPTIONS] [COMMAND_OPTIONS]>>>
+   Usage: <<<hdfs dfs [COMMAND [COMMAND_OPTIONS]]>>>
 
    Run a filesystem command on the file system supported in Hadoop.
    The various COMMAND_OPTIONS can be found at
@@ -66,97 +64,307 @@ HDFS Commands Guide
 
 ** <<<fetchdt>>>
 
-   Gets Delegation Token from a NameNode.
-   See {{{./HdfsUserGuide.html#fetchdt}fetchdt}} for more info.
-
-   Usage: <<<hdfs fetchdt [GENERIC_OPTIONS]
-          [--webservice <namenode_http_addr>] <path> >>>
+   Usage: <<<hdfs fetchdt [--webservice <namenode_http_addr>] <path> >>>
 
 *------------------------------+---------------------------------------------+
 || COMMAND_OPTION              || Description
 *------------------------------+---------------------------------------------+
-| <fileName>                   | File name to store the token into.
-*------------------------------+---------------------------------------------+
 | --webservice <https_address> | use http protocol instead of RPC
 *------------------------------+---------------------------------------------+
+| <fileName>                   | File name to store the token into.
+*------------------------------+---------------------------------------------+
+
+
+   Gets Delegation Token from a NameNode.
+   See {{{./HdfsUserGuide.html#fetchdt}fetchdt}} for more info.
 
 ** <<<fsck>>>
 
-   Runs a HDFS filesystem checking utility.
-   See {{{./HdfsUserGuide.html#fsck}fsck}} for more info.
+   Usage:
 
-   Usage: <<<hdfs fsck [GENERIC_OPTIONS] <path>
-          [-list-corruptfileblocks | 
+---
+   hdfs fsck <path>
+          [-list-corruptfileblocks |
           [-move | -delete | -openforwrite]
           [-files [-blocks [-locations | -racks]]]
-          [-includeSnapshots] [-showprogress]>>>
+          [-includeSnapshots] [-showprogress]
+---
 
 *------------------------+---------------------------------------------+
 ||  COMMAND_OPTION       || Description
 *------------------------+---------------------------------------------+
 |   <path>               | Start checking from this path.
 *------------------------+---------------------------------------------+
-| -move                  | Move corrupted files to /lost+found.
-*------------------------+---------------------------------------------+
 | -delete                | Delete corrupted files.
 *------------------------+---------------------------------------------+
 | -files                 | Print out files being checked.
 *------------------------+---------------------------------------------+
-| -openforwrite          | Print out files opened for write.
+| -files -blocks         | Print out the block report
+*------------------------+---------------------------------------------+
+| -files -blocks -locations | Print out locations for every block.
 *------------------------+---------------------------------------------+
-|                        | Include snapshot data if the given path 
-| -includeSnapshots      | indicates a snapshottable directory or 
+| -files -blocks -racks  | Print out network topology for data-node locations.
+*------------------------+---------------------------------------------+
+|                        | Include snapshot data if the given path
+| -includeSnapshots      | indicates a snapshottable directory or
 |                        | there are snapshottable directories under it.
 *------------------------+---------------------------------------------+
-| -list-corruptfileblocks| Print out list of missing blocks and 
+| -list-corruptfileblocks| Print out list of missing blocks and
 |                        | files they belong to.
 *------------------------+---------------------------------------------+
-| -blocks                | Print out block report.
-*------------------------+---------------------------------------------+
-| -locations             | Print out locations for every block.
+| -move                  | Move corrupted files to /lost+found.
 *------------------------+---------------------------------------------+
-| -racks                 | Print out network topology for data-node locations.
+| -openforwrite          | Print out files opened for write.
 *------------------------+---------------------------------------------+
 | -showprogress          | Print out dots for progress in output. Default is OFF
 |                        | (no progress).
 *------------------------+---------------------------------------------+
 
-** <<<version>>>
 
-   Prints the version.
+   Runs the HDFS filesystem checking utility.
+   See {{{./HdfsUserGuide.html#fsck}fsck}} for more info.
+
+** <<<getconf>>>
+
+   Usage:
+
+---
+   hdfs getconf -namenodes
+   hdfs getconf -secondaryNameNodes
+   hdfs getconf -backupNodes
+   hdfs getconf -includeFile
+   hdfs getconf -excludeFile
+   hdfs getconf -nnRpcAddresses
+   hdfs getconf -confKey [key]
+---
+
+*------------------------+---------------------------------------------+
+||  COMMAND_OPTION       || Description
+*------------------------+---------------------------------------------+
+|	-namenodes			| gets list of namenodes in the cluster.
+*------------------------+---------------------------------------------+
+|	-secondaryNameNodes	| gets list of secondary namenodes in the cluster.
+*------------------------+---------------------------------------------+
+|	-backupNodes		| gets list of backup nodes in the cluster.
+*------------------------+---------------------------------------------+
+|	-includeFile		| gets the include file path that defines the datanodes that can join the cluster.
+*------------------------+---------------------------------------------+
+|	-excludeFile		| gets the exclude file path that defines the datanodes that need to decommissioned.
+*------------------------+---------------------------------------------+
+|	-nnRpcAddresses		| gets the namenode rpc addresses
+*------------------------+---------------------------------------------+
+|	-confKey [key]		| gets a specific key from the configuration
+*------------------------+---------------------------------------------+
+
+ Gets configuration information from the configuration directory, post-processing.
+
+** <<<groups>>>
+
+   Usage: <<<hdfs groups [username ...]>>>
+
+   Returns the group information given one or more usernames.
+
+** <<<lsSnapshottableDir>>>
+
+   Usage: <<<hdfs lsSnapshottableDir [-help]>>>
+
+*------------------------+---------------------------------------------+
+||  COMMAND_OPTION       || Description
+*------------------------+---------------------------------------------+
+| -help | print help
+*------------------------+---------------------------------------------+
+
+  Get the list of snapshottable directories.  When this is run as a super user,
+  it returns all snapshottable directories.  Otherwise it returns those directories
+  that are owned by the current user.
+
+** <<<jmxget>>>
+
+   Usage: <<<hdfs jmxget [-localVM ConnectorURL | -port port | -server mbeanserver | -service service]>>>
+
+*------------------------+---------------------------------------------+
+||  COMMAND_OPTION       || Description
+*------------------------+---------------------------------------------+
+| -help | print help
+*------------------------+---------------------------------------------+
+| -localVM ConnectorURL | connect to the VM on the same machine
+*------------------------+---------------------------------------------+
+| -port <mbean server port>    |  specify mbean server port, if missing
+|                             | it will try to connect to MBean Server in
+|                              |  the same VM
+*------------------------+---------------------------------------------+
+| -service  | specify jmx service, either DataNode or NameNode, the default
+*------------------------+---------------------------------------------+
+
+    Dump JMX information from a service.
+
+** <<<oev>>>
+
+  Usage: <<<hdfs oev [OPTIONS] -i INPUT_FILE -o OUTPUT_FILE>>>
+
+*** Required command line arguments:
+
+*------------------------+---------------------------------------------+
+||  COMMAND_OPTION       || Description
+*------------------------+---------------------------------------------+
+|-i,--inputFile <arg> | edits file to process, xml (case
+                       | insensitive) extension means XML format,
+                       | any other filename means binary format
+*------------------------+---------------------------------------------+
+| -o,--outputFile <arg> | Name of output file. If the specified
+                      | file exists, it will be overwritten,
+                      | format of the file is determined
+                      | by -p option
+*------------------------+---------------------------------------------+
+
+*** Optional command line arguments:
+
+*------------------------+---------------------------------------------+
+||  COMMAND_OPTION       || Description
+*------------------------+---------------------------------------------+
+| -f,--fix-txids        | Renumber the transaction IDs in the input,
+                      | so that there are no gaps or invalid transaction IDs.
+*------------------------+---------------------------------------------+
+| -h,--help             | Display usage information and exit
+*------------------------+---------------------------------------------+
+| -r,--recover          | When reading binary edit logs, use recovery
+                      | mode.  This will give you the chance to skip
+                      | corrupt parts of the edit log.
+*------------------------+---------------------------------------------+
+| -p,--processor <arg>  | Select which type of processor to apply
+                      | against image file, currently supported
+                      | processors are: binary (native binary format
+                      | that Hadoop uses), xml (default, XML
+                      | format), stats (prints statistics about
+                      | edits file)
+*------------------------+---------------------------------------------+
+| -v,--verbose          | More verbose output, prints the input and
+                      | output filenames, for processors that write
+                      | to a file, also output to screen. On large
+                      | image files this will dramatically increase
+                      | processing time (default is false).
+*------------------------+---------------------------------------------+
+
+  Hadoop offline edits viewer.
+
+** <<<oiv>>>
+
+  Usage: <<<hdfs oiv [OPTIONS] -i INPUT_FILE>>>
+
+*** Required command line arguments:
+
+*------------------------+---------------------------------------------+
+||  COMMAND_OPTION       || Description
+*------------------------+---------------------------------------------+
+|-i,--inputFile <arg> | edits file to process, xml (case
+                       | insensitive) extension means XML format,
+                       | any other filename means binary format
+*------------------------+---------------------------------------------+
+
+*** Optional command line arguments:
+
+*------------------------+---------------------------------------------+
+||  COMMAND_OPTION       || Description
+*------------------------+---------------------------------------------+
+| -h,--help             | Display usage information and exit
+*------------------------+---------------------------------------------+
+| -o,--outputFile <arg> | Name of output file. If the specified
+                      | file exists, it will be overwritten,
+                      | format of the file is determined
+                      | by -p option
+*------------------------+---------------------------------------------+
+| -p,--processor <arg>  | Select which type of processor to apply
+                      | against image file, currently supported
+                      | processors are: binary (native binary format
+                      | that Hadoop uses), xml (default, XML
+                      | format), stats (prints statistics about
+                      | edits file)
+*------------------------+---------------------------------------------+
+
+  Hadoop Offline Image Viewer for newer image files.
+
+** <<<oiv_legacy>>>
+
+  Usage: <<<hdfs oiv_legacy [OPTIONS] -i INPUT_FILE -o OUTPUT_FILE>>>
+
+*------------------------+---------------------------------------------+
+||  COMMAND_OPTION       || Description
+*------------------------+---------------------------------------------+
+| -h,--help             | Display usage information and exit
+*------------------------+---------------------------------------------+
+|-i,--inputFile <arg> | edits file to process, xml (case
+                       | insensitive) extension means XML format,
+                       | any other filename means binary format
+*------------------------+---------------------------------------------+
+| -o,--outputFile <arg> | Name of output file. If the specified
+                      | file exists, it will be overwritten,
+                      | format of the file is determined
+                      | by -p option
+*------------------------+---------------------------------------------+
+
+  Hadoop offline image viewer for older versions of Hadoop.
+
+
+** <<<snapshotDiff>>>
+
+    Usage:  <<<hdfs snapshotDiff <path> <fromSnapshot> <toSnapshot> >>>
+
+    Determine the difference between HDFS snapshots. See the
+    {{{./HdfsSnapshots.html#Get_Snapshots_Difference_Report}HDFS Snapshot Documentation}} for more information.
+
+** <<<version>>>
 
    Usage: <<<hdfs version>>>
 
+   Prints the version.
+
 * Administration Commands
 
    Commands useful for administrators of a hadoop cluster.
 
 ** <<<balancer>>>
 
-   Runs a cluster balancing utility. An administrator can simply press Ctrl-C
-   to stop the rebalancing process. See
-   {{{./HdfsUserGuide.html#Balancer}Balancer}} for more details.
-
    Usage: <<<hdfs balancer [-threshold <threshold>] [-policy <policy>]>>>
 
 *------------------------+----------------------------------------------------+
 || COMMAND_OPTION        | Description
 *------------------------+----------------------------------------------------+
-| -threshold <threshold> | Percentage of disk capacity. This overwrites the
-|                        | default threshold.
-*------------------------+----------------------------------------------------+
 | -policy <policy>       | <<<datanode>>> (default): Cluster is balanced if
 |                        | each datanode is balanced. \
 |                        | <<<blockpool>>>: Cluster is balanced if each block
 |                        | pool in each datanode is balanced.
 *------------------------+----------------------------------------------------+
+| -threshold <threshold> | Percentage of disk capacity. This overwrites the
+|                        | default threshold.
+*------------------------+----------------------------------------------------+
+
+   Runs a cluster balancing utility. An administrator can simply press Ctrl-C
+   to stop the rebalancing process. See
+   {{{./HdfsUserGuide.html#Balancer}Balancer}} for more details.
 
    Note that the <<<blockpool>>> policy is more strict than the <<<datanode>>>
    policy.
 
-** <<<datanode>>>
+** <<<cacheadmin>>>
+
+    Usage: <<<hdfs cacheadmin -addDirective -path <path> -pool <pool-name> [-force] [-replication <replication>] [-ttl <time-to-live>]>>>
+
+    See the {{{./CentralizedCacheManagement.html#cacheadmin_command-line_interface}HDFS Cache Administration Documentation}} for more information.
+
+** <<<crypto>>>
+
+    Usage:
+
+---
+  hdfs crypto -createZone -keyName <keyName> -path <path>
+  hdfs crypto -help <command-name>
+  hdfs crypto -listZones
+---
+
+  See the {{{./TransparentEncryption.html#crypto_command-line_interface}HDFS Transparent Encryption Documentation}} for more information.
 
-   Runs a HDFS datanode.
+
+** <<<datanode>>>
 
    Usage: <<<hdfs datanode [-regular | -rollback | -rollingupgrace rollback]>>>
 
@@ -172,12 +380,14 @@ HDFS Commands Guide
 | -rollingupgrade rollback | Rollback a rolling upgrade operation.
 *-----------------+-----------------------------------------------------------+
 
+   Runs a HDFS datanode.
+
 ** <<<dfsadmin>>>
 
-   Runs a HDFS dfsadmin client.
+  Usage:
 
-+------------------------------------------+
-   Usage: hdfs dfsadmin [GENERIC_OPTIONS]
+------------------------------------------
+    hdfs dfsadmin [GENERIC_OPTIONS]
           [-report [-live] [-dead] [-decommissioning]]
           [-safemode enter | leave | get | wait]
           [-saveNamespace]
@@ -210,7 +420,7 @@ HDFS Commands Guide
           [-getDatanodeInfo <datanode_host:ipc_port>]
           [-triggerBlockReport [-incremental] <datanode_host:ipc_port>]
           [-help [cmd]]
-+------------------------------------------+
+------------------------------------------
 
 *-----------------+-----------------------------------------------------------+
 || COMMAND_OPTION || Description
@@ -323,11 +533,11 @@ HDFS Commands Guide
 *-----------------+-----------------------------------------------------------+
 | -allowSnapshot \<snapshotDir\> | Allowing snapshots of a directory to be
                   | created. If the operation completes successfully, the
-                  | directory becomes snapshottable.
+                  | directory becomes snapshottable. See the {{{./HdfsSnapshots.html}HDFS Snapshot Documentation}} for more information.
 *-----------------+-----------------------------------------------------------+
 | -disallowSnapshot \<snapshotDir\> | Disallowing snapshots of a directory to
                   | be created. All snapshots of the directory must be deleted
-                  | before disallowing snapshots.
+                  | before disallowing snapshots. See the {{{./HdfsSnapshots.html}HDFS Snapshot Documentation}} for more information.
 *-----------------+-----------------------------------------------------------+
 | -fetchImage \<local directory\> | Downloads the most recent fsimage from the
                   | NameNode and saves it in the specified local directory.
@@ -351,30 +561,68 @@ HDFS Commands Guide
                   | is specified.
 *-----------------+-----------------------------------------------------------+
 
-** <<<mover>>>
+   Runs a HDFS dfsadmin client.
 
-   Runs the data migration utility.
-   See {{{./ArchivalStorage.html#Mover_-_A_New_Data_Migration_Tool}Mover}} for more details.
+** <<<haadmin>>>
 
-   Usage: <<<hdfs mover [-p <files/dirs> | -f <local file name>]>>>
+    Usage:
+    
+---
+    hdfs haadmin -checkHealth <serviceId>
+    hdfs haadmin -failover [--forcefence] [--forceactive] <serviceId> <serviceId>
+    hdfs haadmin -getServiceState <serviceId>
+    hdfs haadmin -help <command>
+    hdfs haadmin -transitionToActive <serviceId> [--forceactive]
+    hdfs haadmin -transitionToStandby <serviceId>
+---
 
 *--------------------+--------------------------------------------------------+
 || COMMAND_OPTION    || Description
 *--------------------+--------------------------------------------------------+
-| -p \<files/dirs\>  | Specify a space separated list of HDFS files/dirs to migrate.
+| -checkHealth | check the health of the given NameNode
+*--------------------+--------------------------------------------------------+
+| -failover | initiate a failover between two NameNodes
+*--------------------+--------------------------------------------------------+
+| -getServiceState | determine whether the given NameNode is Active or Standby
+*--------------------+--------------------------------------------------------+
+| -transitionToActive  | transition the state of the given NameNode to Active (Warning: No fencing is done)
+*--------------------+--------------------------------------------------------+
+| -transitionToStandby | transition the state of the given NameNode to Standby (Warning: No fencing is done)
+*--------------------+--------------------------------------------------------+
+
+  See {{{./HDFSHighAvailabilityWithNFS.html#Administrative_commands}HDFS HA with NFS}} or
+  {{{./HDFSHighAvailabilityWithQJM.html#Administrative_commands}HDFS HA with QJM}} for more
+  information on this command.
+
+** <<<journalnode>>>
+
+   Usage: <<<hdfs journalnode>>>
+
+   This comamnd starts a journalnode for use with {{{./HDFSHighAvailabilityWithQJM.html#Administrative_commands}HDFS HA with QJM}}.
+
+** <<<mover>>>
+
+   Usage: <<<hdfs mover [-p <files/dirs> | -f <local file name>]>>>
+
+*--------------------+--------------------------------------------------------+
+|| COMMAND_OPTION    || Description
 *--------------------+--------------------------------------------------------+
 | -f \<local file\>  | Specify a local file containing a list of HDFS files/dirs to migrate.
 *--------------------+--------------------------------------------------------+
+| -p \<files/dirs\>  | Specify a space separated list of HDFS files/dirs to migrate.
+*--------------------+--------------------------------------------------------+
+
+   Runs the data migration utility.
+   See {{{./ArchivalStorage.html#Mover_-_A_New_Data_Migration_Tool}Mover}} for more details.
 
   Note that, when both -p and -f options are omitted, the default path is the root directory.
 
 ** <<<namenode>>>
 
-   Runs the namenode. More info about the upgrade, rollback and finalize is at
-   {{{./HdfsUserGuide.html#Upgrade_and_Rollback}Upgrade Rollback}}.
+   Usage:
 
-+------------------------------------------+
-   Usage: hdfs namenode [-backup] |
+------------------------------------------
+  hdfs namenode [-backup] |
           [-checkpoint] |
           [-format [-clusterid cid ] [-force] [-nonInteractive] ] |
           [-upgrade [-clusterid cid] [-renameReserved<k-v pairs>] ] |
@@ -387,7 +635,7 @@ HDFS Commands Guide
           [-bootstrapStandby] |
           [-recover [-force] ] |
           [-metadataVersion ]
-+------------------------------------------+
+------------------------------------------
 
 *--------------------+--------------------------------------------------------+
 || COMMAND_OPTION    || Description
@@ -443,11 +691,23 @@ HDFS Commands Guide
                      | metadata versions of the software and the image.
 *--------------------+--------------------------------------------------------+
 
-** <<<secondarynamenode>>>
+   Runs the namenode. More info about the upgrade, rollback and finalize is at
+   {{{./HdfsUserGuide.html#Upgrade_and_Rollback}Upgrade Rollback}}.
 
-   Runs the HDFS secondary namenode.
-   See {{{./HdfsUserGuide.html#Secondary_NameNode}Secondary Namenode}}
-   for more info.
+
+** <<<nfs3>>>
+
+   Usage: <<<hdfs nfs3>>>
+
+   This comamnd starts the NFS3 gateway for use with the {{{./HdfsNfsGateway.html#Start_and_stop_NFS_gateway_service}HDFS NFS3 Service}}.
+
+** <<<portmap>>>
+
+   Usage: <<<hdfs portmap>>>
+
+   This comamnd starts the RPC portmap for use with the {{{./HdfsNfsGateway.html#Start_and_stop_NFS_gateway_service}HDFS NFS3 Service}}.
+
+** <<<secondarynamenode>>>
 
    Usage: <<<hdfs secondarynamenode [-checkpoint [force]] | [-format] |
           [-geteditsize]>>>
@@ -465,6 +725,33 @@ HDFS Commands Guide
                        | the NameNode.
 *----------------------+------------------------------------------------------+
 
+   Runs the HDFS secondary namenode.
+   See {{{./HdfsUserGuide.html#Secondary_NameNode}Secondary Namenode}}
+   for more info.
+
+
+** <<<storagepolicies>>>
+
+    Usage: <<<hdfs storagepolicies>>>
+
+    Lists out all storage policies.  See the {{{./ArchivalStorage.html}HDFS Storage Policy Documentation}} for more information.
+
+** <<<zkfc>>>
+
+
+   Usage: <<<hdfs zkfc [-formatZK [-force] [-nonInteractive]]>>>
+
+*----------------------+------------------------------------------------------+
+|| COMMAND_OPTION      || Description
+*----------------------+------------------------------------------------------+
+| -formatZK | Format the Zookeeper instance
+*----------------------+------------------------------------------------------+
+| -h | Display help
+*----------------------+------------------------------------------------------+
+
+   This comamnd starts a Zookeeper Failover Controller process for use with {{{./HDFSHighAvailabilityWithQJM.html#Administrative_commands}HDFS HA with QJM}}.
+
+
 * Debug Commands
 
    Useful commands to help administrators debug HDFS issues, like validating
@@ -472,30 +759,25 @@ HDFS Commands Guide
 
 ** <<<verify>>>
 
-   Verify HDFS metadata and block files.  If a block file is specified, we
-   will verify that the checksums in the metadata file match the block
-   file.
-
    Usage: <<<hdfs dfs verify [-meta <metadata-file>] [-block <block-file>]>>>
 
 *------------------------+----------------------------------------------------+
 || COMMAND_OPTION        | Description
 *------------------------+----------------------------------------------------+
-| -meta <metadata-file>  | Absolute path for the metadata file on the local file
-|                        | system of the data node.
-*------------------------+----------------------------------------------------+
 | -block <block-file>    | Optional parameter to specify the absolute path for
 |                        | the block file on the local file system of the data
 |                        | node.
 *------------------------+----------------------------------------------------+
+| -meta <metadata-file>  | Absolute path for the metadata file on the local file
+|                        | system of the data node.
+*------------------------+----------------------------------------------------+
 
-
+   Verify HDFS metadata and block files.  If a block file is specified, we
+   will verify that the checksums in the metadata file match the block
+   file.
 
 ** <<<recoverLease>>>
 
-   Recover the lease on the specified path.  The path must reside on an
-   HDFS filesystem.  The default number of retries is 1.
-
    Usage: <<<hdfs dfs recoverLease [-path <path>] [-retries <num-retries>]>>>
 
 *-------------------------------+--------------------------------------------+
@@ -507,3 +789,6 @@ HDFS Commands Guide
 |                               | recoverLease. The default number of retries
 |                               | is 1.
 *-------------------------------+---------------------------------------------+
+
+   Recover the lease on the specified path.  The path must reside on an
+   HDFS filesystem.  The default number of retries is 1.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce011763/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithNFS.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithNFS.apt.vm b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithNFS.apt.vm
index b404e23..14776b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithNFS.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithNFS.apt.vm
@@ -25,7 +25,7 @@ HDFS High Availability
   This guide provides an overview of the HDFS High Availability (HA) feature and
   how to configure and manage an HA HDFS cluster, using NFS for the shared
   storage required by the NameNodes.
- 
+
   This document assumes that the reader has a general understanding of
   general components and node types in an HDFS cluster. Please refer to the
   HDFS Architecture guide for details.
@@ -44,7 +44,7 @@ HDFS High Availability
   an HDFS cluster. Each cluster had a single NameNode, and if that machine or
   process became unavailable, the cluster as a whole would be unavailable
   until the NameNode was either restarted or brought up on a separate machine.
-  
+
   This impacted the total availability of the HDFS cluster in two major ways:
 
     * In the case of an unplanned event such as a machine crash, the cluster would
@@ -52,7 +52,7 @@ HDFS High Availability
 
     * Planned maintenance events such as software or hardware upgrades on the
       NameNode machine would result in windows of cluster downtime.
-  
+
   The HDFS High Availability feature addresses the above problems by providing
   the option of running two redundant NameNodes in the same cluster in an
   Active/Passive configuration with a hot standby. This allows a fast failover to
@@ -67,7 +67,7 @@ HDFS High Availability
   for all client operations in the cluster, while the Standby is simply acting
   as a slave, maintaining enough state to provide a fast failover if
   necessary.
-  
+
   In order for the Standby node to keep its state synchronized with the Active
   node, the current implementation requires that the two nodes both have access
   to a directory on a shared storage device (eg an NFS mount from a NAS). This
@@ -80,12 +80,12 @@ HDFS High Availability
   a failover, the Standby will ensure that it has read all of the edits from the
   shared storage before promoting itself to the Active state. This ensures that
   the namespace state is fully synchronized before a failover occurs.
-  
+
   In order to provide a fast failover, it is also necessary that the Standby node
   have up-to-date information regarding the location of blocks in the cluster.
   In order to achieve this, the DataNodes are configured with the location of
   both NameNodes, and send block location information and heartbeats to both.
-  
+
   It is vital for the correct operation of an HA cluster that only one of the
   NameNodes be Active at a time. Otherwise, the namespace state would quickly
   diverge between the two, risking data loss or other incorrect results.  In
@@ -116,7 +116,7 @@ HDFS High Availability
     network, and power). Beacuse of this, it is recommended that the shared storage
     server be a high-quality dedicated NAS appliance rather than a simple Linux
     server.
-  
+
   Note that, in an HA cluster, the Standby NameNode also performs checkpoints of
   the namespace state, and thus it is not necessary to run a Secondary NameNode,
   CheckpointNode, or BackupNode in an HA cluster. In fact, to do so would be an
@@ -133,7 +133,7 @@ HDFS High Availability
   The new configuration is designed such that all the nodes in the cluster may
   have the same configuration without the need for deploying different
   configuration files to different machines based on the type of the node.
- 
+
   Like HDFS Federation, HA clusters reuse the <<<nameservice ID>>> to identify a
   single HDFS instance that may in fact consist of multiple HA NameNodes. In
   addition, a new abstraction called <<<NameNode ID>>> is added with HA. Each
@@ -330,7 +330,7 @@ HDFS High Availability
       <<dfs_namenode_rpc-address>> will contain the RPC address of the target node, even
       though the configuration may specify that variable as
       <<dfs.namenode.rpc-address.ns1.nn1>>.
-      
+
       Additionally, the following variables referring to the target node to be fenced
       are also available:
 
@@ -345,7 +345,7 @@ HDFS High Availability
 *-----------------------:-----------------------------------+
 | $target_namenodeid    | the namenode ID of the NN to be fenced |
 *-----------------------:-----------------------------------+
-      
+
       These environment variables may also be used as substitutions in the shell
       command itself. For example:
 
@@ -355,7 +355,7 @@ HDFS High Availability
   <value>shell(/path/to/my/script.sh --nameservice=$target_nameserviceid $target_host:$target_port)</value>
 </property>
 ---
-      
+
       If the shell command returns an exit
       code of 0, the fencing is determined to be successful. If it returns any other
       exit code, the fencing was not successful and the next fencing method in the
@@ -386,7 +386,7 @@ HDFS High Availability
 
     * If you are setting up a fresh HDFS cluster, you should first run the format
     command (<hdfs namenode -format>) on one of NameNodes.
-  
+
     * If you have already formatted the NameNode, or are converting a
     non-HA-enabled cluster to be HA-enabled, you should now copy over the
     contents of your NameNode metadata directories to the other, unformatted
@@ -394,7 +394,7 @@ HDFS High Availability
     unformatted NameNode. Running this command will also ensure that the shared
     edits directory (as configured by <<dfs.namenode.shared.edits.dir>>) contains
     sufficient edits transactions to be able to start both NameNodes.
-  
+
     * If you are converting a non-HA NameNode to be HA, you should run the
     command "<hdfs -initializeSharedEdits>", which will initialize the shared
     edits directory with the edits data from the local NameNode edits directories.
@@ -484,7 +484,7 @@ Usage: DFSHAAdmin [-ns <nameserviceId>]
   of coordination data, notifying clients of changes in that data, and
   monitoring clients for failures. The implementation of automatic HDFS failover
   relies on ZooKeeper for the following things:
-  
+
     * <<Failure detection>> - each of the NameNode machines in the cluster
     maintains a persistent session in ZooKeeper. If the machine crashes, the
     ZooKeeper session will expire, notifying the other NameNode that a failover
@@ -585,7 +585,7 @@ Usage: DFSHAAdmin [-ns <nameserviceId>]
   from one of the NameNode hosts.
 
 ----
-$ hdfs zkfc -formatZK
+[hdfs]$ $HADOOP_PREFIX/bin/zkfc -formatZK
 ----
 
   This will create a znode in ZooKeeper inside of which the automatic failover
@@ -605,7 +605,7 @@ $ hdfs zkfc -formatZK
   can start the daemon by running:
 
 ----
-$ hadoop-daemon.sh start zkfc
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon start zkfc
 ----
 
 ** Securing access to ZooKeeper
@@ -646,7 +646,7 @@ digest:hdfs-zkfcs:mypassword
   a command like the following:
 
 ----
-$ java -cp $ZK_HOME/lib/*:$ZK_HOME/zookeeper-3.4.2.jar org.apache.zookeeper.server.auth.DigestAuthenticationProvider hdfs-zkfcs:mypassword
+[hdfs]$ java -cp $ZK_HOME/lib/*:$ZK_HOME/zookeeper-3.4.2.jar org.apache.zookeeper.server.auth.DigestAuthenticationProvider hdfs-zkfcs:mypassword
 output: hdfs-zkfcs:mypassword->hdfs-zkfcs:P/OQvnYyU/nF/mGYvB/xurX8dYs=
 ----
 
@@ -726,24 +726,24 @@ digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
   using the same <<<hdfs haadmin>>> command. It will perform a coordinated
   failover.
 
- 
+
 * BookKeeper as a Shared storage (EXPERIMENTAL)
 
-   One option for shared storage for the NameNode is BookKeeper. 
+   One option for shared storage for the NameNode is BookKeeper.
   BookKeeper achieves high availability and strong durability guarantees by replicating
-  edit log entries across multiple storage nodes. The edit log can be striped across 
-  the storage nodes for high performance. Fencing is supported in the protocol, i.e, 
+  edit log entries across multiple storage nodes. The edit log can be striped across
+  the storage nodes for high performance. Fencing is supported in the protocol, i.e,
   BookKeeper will not allow two writers to write the single edit log.
 
   The meta data for BookKeeper is stored in ZooKeeper.
   In current HA architecture, a Zookeeper cluster is required for ZKFC. The same cluster can be
   for BookKeeper metadata.
 
-  For more details on building a BookKeeper cluster, please refer to the 
+  For more details on building a BookKeeper cluster, please refer to the
    {{{http://zookeeper.apache.org/bookkeeper/docs/trunk/bookkeeperConfig.html }BookKeeper documentation}}
 
  The BookKeeperJournalManager is an implementation of the HDFS JournalManager interface, which allows custom write ahead logging implementations to be plugged into the HDFS NameNode.
- 
+
  **<<BookKeeper Journal Manager>>
 
    To use BookKeeperJournalManager, add the following to hdfs-site.xml.
@@ -772,12 +772,12 @@ digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
    classpath. We explain how to generate a jar file with the journal manager and
    its dependencies, and how to put it into the classpath below.
 
- *** <<More configuration options>> 
+ *** <<More configuration options>>
 
-     * <<dfs.namenode.bookkeeperjournal.output-buffer-size>> - 
+     * <<dfs.namenode.bookkeeperjournal.output-buffer-size>> -
        Number of bytes a bookkeeper journal stream will buffer before
        forcing a flush. Default is 1024.
-     
+
 ----
        <property>
          <name>dfs.namenode.bookkeeperjournal.output-buffer-size</name>
@@ -785,7 +785,7 @@ digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
        </property>
 ----
 
-     * <<dfs.namenode.bookkeeperjournal.ensemble-size>> - 
+     * <<dfs.namenode.bookkeeperjournal.ensemble-size>> -
        Number of bookkeeper servers in edit log ensembles. This
        is the number of bookkeeper servers which need to be available
        for the edit log to be writable. Default is 3.
@@ -797,7 +797,7 @@ digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
        </property>
 ----
 
-     * <<dfs.namenode.bookkeeperjournal.quorum-size>> - 
+     * <<dfs.namenode.bookkeeperjournal.quorum-size>> -
        Number of bookkeeper servers in the write quorum. This is the
        number of bookkeeper servers which must have acknowledged the
        write of an entry before it is considered written. Default is 2.
@@ -809,7 +809,7 @@ digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
        </property>
 ----
 
-     * <<dfs.namenode.bookkeeperjournal.digestPw>> - 
+     * <<dfs.namenode.bookkeeperjournal.digestPw>> -
        Password to use when creating edit log segments.
 
 ----
@@ -819,9 +819,9 @@ digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
        </property>
 ----
 
-     * <<dfs.namenode.bookkeeperjournal.zk.session.timeout>> - 
+     * <<dfs.namenode.bookkeeperjournal.zk.session.timeout>> -
        Session timeout for Zookeeper client from BookKeeper Journal Manager.
-       Hadoop recommends that this value should be less than the ZKFC 
+       Hadoop recommends that this value should be less than the ZKFC
        session timeout value. Default value is 3000.
 
 ----
@@ -838,22 +838,22 @@ digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
 
      $ mvn clean package -Pdist
 
-     This will generate a jar with the BookKeeperJournalManager, 
+     This will generate a jar with the BookKeeperJournalManager,
      hadoop-hdfs/src/contrib/bkjournal/target/hadoop-hdfs-bkjournal-<VERSION>.jar
 
      Note that the -Pdist part of the build command is important, this would
-     copy the dependent bookkeeper-server jar under 
+     copy the dependent bookkeeper-server jar under
      hadoop-hdfs/src/contrib/bkjournal/target/lib.
 
  *** <<Putting the BookKeeperJournalManager in the NameNode classpath>>
 
     To run a HDFS namenode using BookKeeper as a backend, copy the bkjournal and
-    bookkeeper-server jar, mentioned above, into the lib directory of hdfs. In the 
+    bookkeeper-server jar, mentioned above, into the lib directory of hdfs. In the
     standard distribution of HDFS, this is at $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/
 
     cp hadoop-hdfs/src/contrib/bkjournal/target/hadoop-hdfs-bkjournal-<VERSION>.jar $HADOOP_HDFS_HOME/share/hadoop/hdfs/lib/
 
- *** <<Current limitations>> 
+ *** <<Current limitations>>
 
       1) Security in BookKeeper. BookKeeper does not support SASL nor SSL for
          connections between the NameNode and BookKeeper storage nodes.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce011763/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithQJM.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithQJM.apt.vm b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithQJM.apt.vm
index ff6a42c..a3a9f12 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithQJM.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HDFSHighAvailabilityWithQJM.apt.vm
@@ -25,7 +25,7 @@ HDFS High Availability Using the Quorum Journal Manager
   This guide provides an overview of the HDFS High Availability (HA) feature
   and how to configure and manage an HA HDFS cluster, using the Quorum Journal
   Manager (QJM) feature.
- 
+
   This document assumes that the reader has a general understanding of
   general components and node types in an HDFS cluster. Please refer to the
   HDFS Architecture guide for details.
@@ -44,7 +44,7 @@ HDFS High Availability Using the Quorum Journal Manager
   an HDFS cluster. Each cluster had a single NameNode, and if that machine or
   process became unavailable, the cluster as a whole would be unavailable
   until the NameNode was either restarted or brought up on a separate machine.
-  
+
   This impacted the total availability of the HDFS cluster in two major ways:
 
     * In the case of an unplanned event such as a machine crash, the cluster would
@@ -52,7 +52,7 @@ HDFS High Availability Using the Quorum Journal Manager
 
     * Planned maintenance events such as software or hardware upgrades on the
       NameNode machine would result in windows of cluster downtime.
-  
+
   The HDFS High Availability feature addresses the above problems by providing
   the option of running two redundant NameNodes in the same cluster in an
   Active/Passive configuration with a hot standby. This allows a fast failover to
@@ -67,7 +67,7 @@ HDFS High Availability Using the Quorum Journal Manager
   for all client operations in the cluster, while the Standby is simply acting
   as a slave, maintaining enough state to provide a fast failover if
   necessary.
-  
+
   In order for the Standby node to keep its state synchronized with the Active
   node, both nodes communicate with a group of separate daemons called
   "JournalNodes" (JNs). When any namespace modification is performed by the
@@ -78,12 +78,12 @@ HDFS High Availability Using the Quorum Journal Manager
   failover, the Standby will ensure that it has read all of the edits from the
   JounalNodes before promoting itself to the Active state. This ensures that the
   namespace state is fully synchronized before a failover occurs.
-  
+
   In order to provide a fast failover, it is also necessary that the Standby node
   have up-to-date information regarding the location of blocks in the cluster.
   In order to achieve this, the DataNodes are configured with the location of
   both NameNodes, and send block location information and heartbeats to both.
-  
+
   It is vital for the correct operation of an HA cluster that only one of the
   NameNodes be Active at a time. Otherwise, the namespace state would quickly
   diverge between the two, risking data loss or other incorrect results.  In
@@ -113,7 +113,7 @@ HDFS High Availability Using the Quorum Journal Manager
     you should run an odd number of JNs, (i.e. 3, 5, 7, etc.). Note that when
     running with N JournalNodes, the system can tolerate at most (N - 1) / 2
     failures and continue to function normally.
-  
+
   Note that, in an HA cluster, the Standby NameNode also performs checkpoints of
   the namespace state, and thus it is not necessary to run a Secondary NameNode,
   CheckpointNode, or BackupNode in an HA cluster. In fact, to do so would be an
@@ -130,7 +130,7 @@ HDFS High Availability Using the Quorum Journal Manager
   The new configuration is designed such that all the nodes in the cluster may
   have the same configuration without the need for deploying different
   configuration files to different machines based on the type of the node.
- 
+
   Like HDFS Federation, HA clusters reuse the <<<nameservice ID>>> to identify a
   single HDFS instance that may in fact consist of multiple HA NameNodes. In
   addition, a new abstraction called <<<NameNode ID>>> is added with HA. Each
@@ -347,7 +347,7 @@ HDFS High Availability Using the Quorum Journal Manager
       <<dfs_namenode_rpc-address>> will contain the RPC address of the target node, even
       though the configuration may specify that variable as
       <<dfs.namenode.rpc-address.ns1.nn1>>.
-      
+
       Additionally, the following variables referring to the target node to be fenced
       are also available:
 
@@ -362,7 +362,7 @@ HDFS High Availability Using the Quorum Journal Manager
 *-----------------------:-----------------------------------+
 | $target_namenodeid    | the namenode ID of the NN to be fenced |
 *-----------------------:-----------------------------------+
-      
+
       These environment variables may also be used as substitutions in the shell
       command itself. For example:
 
@@ -372,7 +372,7 @@ HDFS High Availability Using the Quorum Journal Manager
   <value>shell(/path/to/my/script.sh --nameservice=$target_nameserviceid $target_host:$target_port)</value>
 </property>
 ---
-      
+
       If the shell command returns an exit
       code of 0, the fencing is determined to be successful. If it returns any other
       exit code, the fencing was not successful and the next fencing method in the
@@ -424,7 +424,7 @@ HDFS High Availability Using the Quorum Journal Manager
 
     * If you are setting up a fresh HDFS cluster, you should first run the format
     command (<hdfs namenode -format>) on one of NameNodes.
-  
+
     * If you have already formatted the NameNode, or are converting a
     non-HA-enabled cluster to be HA-enabled, you should now copy over the
     contents of your NameNode metadata directories to the other, unformatted
@@ -432,7 +432,7 @@ HDFS High Availability Using the Quorum Journal Manager
     unformatted NameNode. Running this command will also ensure that the
     JournalNodes (as configured by <<dfs.namenode.shared.edits.dir>>) contain
     sufficient edits transactions to be able to start both NameNodes.
-  
+
     * If you are converting a non-HA NameNode to be HA, you should run the
     command "<hdfs -initializeSharedEdits>", which will initialize the
     JournalNodes with the edits data from the local NameNode edits directories.
@@ -522,7 +522,7 @@ Usage: DFSHAAdmin [-ns <nameserviceId>]
   of coordination data, notifying clients of changes in that data, and
   monitoring clients for failures. The implementation of automatic HDFS failover
   relies on ZooKeeper for the following things:
-  
+
     * <<Failure detection>> - each of the NameNode machines in the cluster
     maintains a persistent session in ZooKeeper. If the machine crashes, the
     ZooKeeper session will expire, notifying the other NameNode that a failover
@@ -623,7 +623,7 @@ Usage: DFSHAAdmin [-ns <nameserviceId>]
   from one of the NameNode hosts.
 
 ----
-$ hdfs zkfc -formatZK
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs zkfc -formatZK
 ----
 
   This will create a znode in ZooKeeper inside of which the automatic failover
@@ -643,7 +643,7 @@ $ hdfs zkfc -formatZK
   can start the daemon by running:
 
 ----
-$ hadoop-daemon.sh start zkfc
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon start zkfc
 ----
 
 ** Securing access to ZooKeeper
@@ -684,7 +684,7 @@ digest:hdfs-zkfcs:mypassword
   a command like the following:
 
 ----
-$ java -cp $ZK_HOME/lib/*:$ZK_HOME/zookeeper-3.4.2.jar org.apache.zookeeper.server.auth.DigestAuthenticationProvider hdfs-zkfcs:mypassword
+[hdfs]$ java -cp $ZK_HOME/lib/*:$ZK_HOME/zookeeper-3.4.2.jar org.apache.zookeeper.server.auth.DigestAuthenticationProvider hdfs-zkfcs:mypassword
 output: hdfs-zkfcs:mypassword->hdfs-zkfcs:P/OQvnYyU/nF/mGYvB/xurX8dYs=
 ----
 
@@ -786,18 +786,18 @@ digest:hdfs-zkfcs:vlUvLnd8MlacsE80rDuu6ONESbM=:rwcda
     operations, the operation will fail.
 
     [[3]] Start one of the NNs with the <<<'-upgrade'>>> flag.
-  
+
     [[4]] On start, this NN will not enter the standby state as usual in an HA
     setup. Rather, this NN will immediately enter the active state, perform an
     upgrade of its local storage dirs, and also perform an upgrade of the shared
     edit log.
-  
+
     [[5]] At this point the other NN in the HA pair will be out of sync with
     the upgraded NN. In order to bring it back in sync and once again have a highly
     available setup, you should re-bootstrap this NameNode by running the NN with
     the <<<'-bootstrapStandby'>>> flag. It is an error to start this second NN with
     the <<<'-upgrade'>>> flag.
-  
+
   Note that if at any time you want to restart the NameNodes before finalizing
   or rolling back the upgrade, you should start the NNs as normal, i.e. without
   any special startup flag.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ce011763/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm
index 152a985..8555b23 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsNfsGateway.apt.vm
@@ -36,15 +36,15 @@ HDFS NFS Gateway
      HDFS file system.
 
    * Users can stream data directly to HDFS through the mount point. File
-     append is supported but random write is not supported. 
+     append is supported but random write is not supported.
 
   The NFS gateway machine needs the same thing to run an HDFS client like Hadoop JAR files, HADOOP_CONF directory.
-  The NFS gateway can be on the same host as DataNode, NameNode, or any HDFS client. 
+  The NFS gateway can be on the same host as DataNode, NameNode, or any HDFS client.
 
 
 * {Configuration}
 
-   The NFS-gateway uses proxy user to proxy all the users accessing the NFS mounts. 
+   The NFS-gateway uses proxy user to proxy all the users accessing the NFS mounts.
    In non-secure mode, the user running the gateway is the proxy user, while in secure mode the
    user in Kerberos keytab is the proxy user. Suppose the proxy user is 'nfsserver'
    and users belonging to the groups 'users-group1'
@@ -57,10 +57,10 @@ HDFS NFS Gateway
   <name>hadoop.proxyuser.nfsserver.groups</name>
   <value>root,users-group1,users-group2</value>
   <description>
-         The 'nfsserver' user is allowed to proxy all members of the 'users-group1' and 
+         The 'nfsserver' user is allowed to proxy all members of the 'users-group1' and
          'users-group2' groups. Note that in most cases you will need to include the
          group "root" because the user "root" (which usually belonges to "root" group) will
-         generally be the user that initially executes the mount on the NFS client system. 
+         generally be the user that initially executes the mount on the NFS client system.
          Set this to '*' to allow nfsserver user to proxy any group.
   </description>
 </property>
@@ -78,7 +78,7 @@ HDFS NFS Gateway
 ----
 
    The above are the only required configuration for the NFS gateway in non-secure mode. For Kerberized
-   hadoop clusters, the following configurations need to be added to hdfs-site.xml for the gateway (NOTE: replace 
+   hadoop clusters, the following configurations need to be added to hdfs-site.xml for the gateway (NOTE: replace
    string "nfsserver" with the proxy user name and ensure the user contained in the keytab is
    also the same proxy user):
 
@@ -95,7 +95,7 @@ HDFS NFS Gateway
     <value>nfsserver/_HOST@YOUR-REALM.COM</value>
   </property>
 ----
-  
+
    The rest of the NFS gateway configurations are optional for both secure and non-secure mode.
 
    The AIX NFS client has a {{{https://issues.apache.org/jira/browse/HDFS-6549}few known issues}}
@@ -119,11 +119,11 @@ HDFS NFS Gateway
 
    It's strongly recommended for the users to update a few configuration properties based on their use
    cases. All the following configuration properties can be added or updated in hdfs-site.xml.
-  
-   * If the client mounts the export with access time update allowed, make sure the following 
-    property is not disabled in the configuration file. Only NameNode needs to restart after 
+
+   * If the client mounts the export with access time update allowed, make sure the following
+    property is not disabled in the configuration file. Only NameNode needs to restart after
     this property is changed. On some Unix systems, the user can disable access time update
-    by mounting the export with "noatime". If the export is mounted with "noatime", the user 
+    by mounting the export with "noatime". If the export is mounted with "noatime", the user
     doesn't need to change the following property and thus no need to restart namenode.
 
 ----
@@ -149,11 +149,11 @@ HDFS NFS Gateway
       this property is updated.
 
 ----
-  <property>    
+  <property>
     <name>nfs.dump.dir</name>
     <value>/tmp/.hdfs-nfs</value>
   </property>
----- 
+----
 
   * 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
@@ -161,7 +161,7 @@ HDFS NFS Gateway
     characters. The machine name format can be a single host, 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 
+    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.
 
 ----
@@ -171,22 +171,22 @@ HDFS NFS Gateway
 </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 
+  * 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
    to add the following. Note, debug trace, especially for ONCRPC, can be very verbose.
 
     To change logging level:
 
------------------------------------------------ 
+-----------------------------------------------
     log4j.logger.org.apache.hadoop.hdfs.nfs=DEBUG
------------------------------------------------ 
+-----------------------------------------------
 
     To get more details of ONCRPC requests:
 
------------------------------------------------ 
+-----------------------------------------------
     log4j.logger.org.apache.hadoop.oncrpc=DEBUG
------------------------------------------------ 
+-----------------------------------------------
 
 
 * {Start and stop NFS gateway service}
@@ -195,53 +195,39 @@ HDFS NFS Gateway
   The NFS gateway process has both nfsd and mountd. It shares the HDFS root "/" as the
   only export. It is recommended to use the portmap included in NFS gateway package. Even
   though NFS gateway works with portmap/rpcbind provide by most Linux distributions, the
-  package included portmap is needed on some Linux systems such as REHL6.2 due to an 
+  package included portmap is needed on some Linux systems such as REHL6.2 due to an
   {{{https://bugzilla.redhat.com/show_bug.cgi?id=731542}rpcbind bug}}. More detailed discussions can
   be found in {{{https://issues.apache.org/jira/browse/HDFS-4763}HDFS-4763}}.
 
-   [[1]] Stop nfs/rpcbind/portmap services provided by the platform (commands can be different on various Unix platforms):
-      
+   [[1]] Stop nfsv3 and rpcbind/portmap services provided by the platform (commands can be different on various Unix platforms):
+
 -------------------------
-     service nfs stop
-      
-     service rpcbind stop
+[root]> service nfs stop
+[root]> service rpcbind stop
 -------------------------
 
-
-   [[2]] Start package included portmap (needs root privileges):
+   [[2]] Start Hadoop's portmap (needs root privileges):
 
 -------------------------
-     hdfs portmap
-  
-     OR
-
-     hadoop-daemon.sh start portmap
+[root]> $HADOOP_PREFIX/bin/hdfs --daemon start portmap
 -------------------------
 
    [[3]] Start mountd and nfsd.
-   
+
      No root privileges are required for this command. In non-secure mode, the NFS gateway
-     should be started by the proxy user mentioned at the beginning of this user guide. 
-     While in secure mode, any user can start NFS gateway 
+     should be started by the proxy user mentioned at the beginning of this user guide.
+     While in secure mode, any user can start NFS gateway
      as long as the user has read access to the Kerberos keytab defined in "nfs.keytab.file".
 
 -------------------------
-     hdfs nfs3
-
-     OR
-
-     hadoop-daemon.sh start nfs3
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon start nfs3
 -------------------------
 
-     Note, if the hadoop-daemon.sh script starts the NFS gateway, its log can be found in the hadoop log folder.
-
-
    [[4]] Stop NFS gateway services.
 
 -------------------------
-      hadoop-daemon.sh stop nfs3
-
-      hadoop-daemon.sh stop portmap
+[hdfs]$ $HADOOP_PREFIX/bin/hdfs --daemon stop nfs3
+[root]> $HADOOP_PREFIX/bin/hdfs --daemon stop portmap
 -------------------------
 
   Optionally, you can forgo running the Hadoop-provided portmap daemon and
@@ -263,7 +249,7 @@ HDFS NFS Gateway
     [[1]] Execute the following command to verify if all the services are up and running:
 
 -------------------------
-       rpcinfo -p $nfs_server_ip
+[root]> rpcinfo -p $nfs_server_ip
 -------------------------
 
      You should see output similar to the following:
@@ -293,11 +279,11 @@ HDFS NFS Gateway
     [[2]]  Verify if the HDFS namespace is exported and can be mounted.
 
 -------------------------
-        showmount -e $nfs_server_ip                         
+[root]> showmount -e $nfs_server_ip
 -------------------------
 
       You should see output similar to the following:
-     
+
 -------------------------
         Exports list on $nfs_server_ip :
 
@@ -307,22 +293,22 @@ HDFS NFS Gateway
 
 * {Mount the export “/”}
 
-  Currently NFS v3 only uses TCP as the transportation protocol. 
+  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 
+  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 
+
+  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:
 
--------------------------------------------------------------------  
-       mount -t nfs -o vers=3,proto=tcp,nolock,noacl $server:/  $mount_point
+-------------------------------------------------------------------
+ [root]>mount -t nfs -o vers=3,proto=tcp,nolock,noacl $server:/  $mount_point
 -------------------------------------------------------------------
 
-  Then the users can access HDFS as part of the local file system except that, 
+  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
@@ -347,7 +333,7 @@ HDFS NFS Gateway
 * {User authentication and mapping}
 
   NFS gateway in this release uses AUTH_UNIX style authentication. When the user on NFS client
-  accesses the mount point, NFS client passes the UID to NFS gateway. 
+  accesses the mount point, NFS client passes the UID to NFS gateway.
   NFS gateway does a lookup to find user name from the UID, and then passes the
   username to the HDFS along with the HDFS requests.
   For example, if the NFS client has current user as "admin", when the user accesses
@@ -358,7 +344,7 @@ HDFS NFS Gateway
   The system administrator must ensure that the user on NFS client host has the same
   name and UID as that on the NFS gateway host. This is usually not a problem if
   the same user management system (e.g., LDAP/NIS) is used to create and deploy users on
-  HDFS nodes and NFS client node. In case the user account is created manually on different hosts, one might need to 
+  HDFS nodes and NFS client node. In case the user account is created manually on different hosts, one might need to
   modify UID (e.g., do "usermod -u 123 myusername") on either NFS client or NFS gateway host
   in order to make it the same on both sides. More technical details of RPC AUTH_UNIX can be found
   in {{{http://tools.ietf.org/html/rfc1057}RPC specification}}.


[06/25] hadoop git commit: HDFS-7585. Get TestEnhancedByteBufferAccess working on CPU architectures with page sizes other than 4096 (Sam Liu via Colin P. McCabe)

Posted by zj...@apache.org.
HDFS-7585. Get TestEnhancedByteBufferAccess working on CPU architectures with page sizes other than 4096 (Sam Liu via Colin P. McCabe)


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

Branch: refs/heads/YARN-2928
Commit: 446545c496fdab75e76c8124c98324e37150b5dc
Parents: d15cbae
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jan 14 11:00:20 2015 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Wed Jan 14 11:13:55 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  3 +
 .../hadoop/fs/TestEnhancedByteBufferAccess.java | 66 ++++++++++----------
 2 files changed, 37 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/446545c4/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 259d010..421d2f0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -691,6 +691,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7470. SecondaryNameNode need twice memory when calling
     reloadFromImageFile. (zhaoyunjiong via cnauroth)
 
+    HDFS-7585. Get TestEnhancedByteBufferAccess working on CPU architectures
+    with page sizes other than 4096 (Sam Liu via Colin P. McCabe)
+
 Release 2.6.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/446545c4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
index 5040a3b..296c8d2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestEnhancedByteBufferAccess.java
@@ -113,7 +113,8 @@ public class TestEnhancedByteBufferAccess {
     return resultArray;
   }
   
-  private static final int BLOCK_SIZE = 4096;
+  private static final int BLOCK_SIZE = 
+      (int) NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize();
   
   public static HdfsConfiguration initZeroCopyTest() {
     Assume.assumeTrue(NativeIO.isAvailable());
@@ -140,7 +141,7 @@ public class TestEnhancedByteBufferAccess {
     MiniDFSCluster cluster = null;
     final Path TEST_PATH = new Path("/a");
     FSDataInputStream fsIn = null;
-    final int TEST_FILE_LENGTH = 12345;
+    final int TEST_FILE_LENGTH = 3 * BLOCK_SIZE;
     
     FileSystem fs = null;
     try {
@@ -163,15 +164,15 @@ public class TestEnhancedByteBufferAccess {
       IOUtils.readFully(fsIn, original, 0, TEST_FILE_LENGTH);
       fsIn.close();
       fsIn = fs.open(TEST_PATH);
-      ByteBuffer result = fsIn.read(null, 4096,
+      ByteBuffer result = fsIn.read(null, BLOCK_SIZE,
           EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(4096, result.remaining());
+      Assert.assertEquals(BLOCK_SIZE, result.remaining());
       HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
-      Assert.assertEquals(4096,
+      Assert.assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalBytesRead());
-      Assert.assertEquals(4096,
+      Assert.assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
-      Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, 4096),
+      Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, BLOCK_SIZE),
           byteBufferToArray(result));
       fsIn.releaseBuffer(result);
     } finally {
@@ -187,7 +188,7 @@ public class TestEnhancedByteBufferAccess {
     MiniDFSCluster cluster = null;
     final Path TEST_PATH = new Path("/a");
     FSDataInputStream fsIn = null;
-    final int TEST_FILE_LENGTH = 12345;
+    final int TEST_FILE_LENGTH = 3 * BLOCK_SIZE;
     
     FileSystem fs = null;
     try {
@@ -210,24 +211,24 @@ public class TestEnhancedByteBufferAccess {
       fsIn.close();
       fsIn = fs.open(TEST_PATH);
 
-      // Try to read 8192, but only get 4096 because of the block size.
+      // Try to read (2 * ${BLOCK_SIZE}), but only get ${BLOCK_SIZE} because of the block size.
       HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
       ByteBuffer result =
-        dfsIn.read(null, 8192, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(4096, result.remaining());
-      Assert.assertEquals(4096,
+        dfsIn.read(null, 2 * BLOCK_SIZE, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
+      Assert.assertEquals(BLOCK_SIZE, result.remaining());
+      Assert.assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalBytesRead());
-      Assert.assertEquals(4096,
+      Assert.assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
-      Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, 4096),
+      Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, BLOCK_SIZE),
           byteBufferToArray(result));
       dfsIn.releaseBuffer(result);
       
-      // Try to read 4097, but only get 4096 because of the block size.
+      // Try to read (1 + ${BLOCK_SIZE}), but only get ${BLOCK_SIZE} because of the block size.
       result = 
-          dfsIn.read(null, 4097, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(4096, result.remaining());
-      Assert.assertArrayEquals(Arrays.copyOfRange(original, 4096, 8192),
+          dfsIn.read(null, 1 + BLOCK_SIZE, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
+      Assert.assertEquals(BLOCK_SIZE, result.remaining());
+      Assert.assertArrayEquals(Arrays.copyOfRange(original, BLOCK_SIZE, 2 * BLOCK_SIZE),
           byteBufferToArray(result));
       dfsIn.releaseBuffer(result);
     } finally {
@@ -243,7 +244,7 @@ public class TestEnhancedByteBufferAccess {
     MiniDFSCluster cluster = null;
     final Path TEST_PATH = new Path("/a");
     FSDataInputStream fsIn = null;
-    final int TEST_FILE_LENGTH = 12345;
+    final int TEST_FILE_LENGTH = 3 * BLOCK_SIZE;
     
     FileSystem fs = null;
     try {
@@ -269,18 +270,18 @@ public class TestEnhancedByteBufferAccess {
       HdfsDataInputStream dfsIn = (HdfsDataInputStream)fsIn;
       ByteBuffer result;
       try {
-        result = dfsIn.read(null, 4097, EnumSet.noneOf(ReadOption.class));
+        result = dfsIn.read(null, BLOCK_SIZE + 1, EnumSet.noneOf(ReadOption.class));
         Assert.fail("expected UnsupportedOperationException");
       } catch (UnsupportedOperationException e) {
         // expected
       }
-      result = dfsIn.read(null, 4096, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-      Assert.assertEquals(4096, result.remaining());
-      Assert.assertEquals(4096,
+      result = dfsIn.read(null, BLOCK_SIZE, EnumSet.of(ReadOption.SKIP_CHECKSUMS));
+      Assert.assertEquals(BLOCK_SIZE, result.remaining());
+      Assert.assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalBytesRead());
-      Assert.assertEquals(4096,
+      Assert.assertEquals(BLOCK_SIZE,
           dfsIn.getReadStatistics().getTotalZeroCopyBytesRead());
-      Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, 4096),
+      Assert.assertArrayEquals(Arrays.copyOfRange(original, 0, BLOCK_SIZE),
           byteBufferToArray(result));
     } finally {
       if (fsIn != null) fsIn.close();
@@ -330,7 +331,7 @@ public class TestEnhancedByteBufferAccess {
     HdfsConfiguration conf = initZeroCopyTest();
     MiniDFSCluster cluster = null;
     final Path TEST_PATH = new Path("/a");
-    final int TEST_FILE_LENGTH = 16385;
+    final int TEST_FILE_LENGTH = 5 * BLOCK_SIZE;
     final int RANDOM_SEED = 23453;
     final String CONTEXT = "testZeroCopyMmapCacheContext";
     FSDataInputStream fsIn = null;
@@ -360,10 +361,10 @@ public class TestEnhancedByteBufferAccess {
     final ShortCircuitCache cache = ClientContext.get(
         CONTEXT, new DFSClient.Conf(conf)). getShortCircuitCache();
     cache.accept(new CountingVisitor(0, 5, 5, 0));
-    results[0] = fsIn.read(null, 4096,
+    results[0] = fsIn.read(null, BLOCK_SIZE,
         EnumSet.of(ReadOption.SKIP_CHECKSUMS));
     fsIn.seek(0);
-    results[1] = fsIn.read(null, 4096,
+    results[1] = fsIn.read(null, BLOCK_SIZE,
         EnumSet.of(ReadOption.SKIP_CHECKSUMS));
 
     // The mmap should be of the first block of the file.
@@ -386,9 +387,9 @@ public class TestEnhancedByteBufferAccess {
     });
 
     // Read more blocks.
-    results[2] = fsIn.read(null, 4096,
+    results[2] = fsIn.read(null, BLOCK_SIZE,
         EnumSet.of(ReadOption.SKIP_CHECKSUMS));
-    results[3] = fsIn.read(null, 4096,
+    results[3] = fsIn.read(null, BLOCK_SIZE,
         EnumSet.of(ReadOption.SKIP_CHECKSUMS));
 
     // we should have 3 mmaps, 1 evictable
@@ -592,7 +593,7 @@ public class TestEnhancedByteBufferAccess {
     BlockReaderTestUtil.enableBlockReaderFactoryTracing();
     BlockReaderTestUtil.enableHdfsCachingTracing();
 
-    final int TEST_FILE_LENGTH = 16385;
+    final int TEST_FILE_LENGTH = BLOCK_SIZE;
     final Path TEST_PATH = new Path("/a");
     final int RANDOM_SEED = 23453;
     HdfsConfiguration conf = initZeroCopyTest();
@@ -601,7 +602,8 @@ public class TestEnhancedByteBufferAccess {
     final String CONTEXT = "testZeroCopyReadOfCachedData";
     conf.set(DFSConfigKeys.DFS_CLIENT_CONTEXT, CONTEXT);
     conf.setLong(DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
-        DFSTestUtil.roundUpToMultiple(TEST_FILE_LENGTH, 4096));
+        DFSTestUtil.roundUpToMultiple(TEST_FILE_LENGTH,
+          (int) NativeIO.POSIX.getCacheManipulator().getOperatingSystemPageSize()));
     MiniDFSCluster cluster = null;
     ByteBuffer result = null, result2 = null;
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();


[19/25] hadoop git commit: HADOOP-8757. Metrics should disallow names with invalid characters (rchiang via rkanter)

Posted by zj...@apache.org.
HADOOP-8757. Metrics should disallow names with invalid characters (rchiang via rkanter)


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

Branch: refs/heads/YARN-2928
Commit: b6ff9c03a4f8aba945e562a7ff60b0fc6a1cd040
Parents: 9e33116
Author: Robert Kanter <rk...@apache.org>
Authored: Thu Jan 15 11:39:43 2015 -0800
Committer: Robert Kanter <rk...@apache.org>
Committed: Thu Jan 15 11:39:43 2015 -0800

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |  3 ++
 .../hadoop/metrics2/lib/MetricsRegistry.java    | 14 +++++++
 .../metrics2/lib/TestMetricsRegistry.java       | 42 ++++++++++++++++++++
 3 files changed, 59 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6ff9c03/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 f80eeab..5cce74b 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -483,6 +483,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11481. ClassCastException while using a key created by keytool to
     create encryption zone. (Charles Lamb via Colin P. Mccabe)
 
+    HADOOP-8757. Metrics should disallow names with invalid characters
+    (rchiang via rkanter)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6ff9c03/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MetricsRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MetricsRegistry.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MetricsRegistry.java
index 1c0d30e..4b561f2 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MetricsRegistry.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MetricsRegistry.java
@@ -363,6 +363,20 @@ public class MetricsRegistry {
   }
 
   private void checkMetricName(String name) {
+    // Check for invalid characters in metric name
+    boolean foundWhitespace = false;
+    for (int i = 0; i < name.length(); i++) {
+      char c = name.charAt(i);
+      if (Character.isWhitespace(c)) {
+        foundWhitespace = true;
+        break;
+      }
+    }
+    if (foundWhitespace) {
+      throw new MetricsException("Metric name '"+ name +
+          "' contains illegal whitespace character");
+    }
+    // Check if name has already been registered
     if (metricsMap.containsKey(name)) {
       throw new MetricsException("Metric name "+ name +" already exists!");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b6ff9c03/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java
index 47b496f..af1ff96 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/metrics2/lib/TestMetricsRegistry.java
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.metrics2.lib;
 
+import org.junit.Ignore;
 import org.junit.Test;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
@@ -57,6 +58,46 @@ public class TestMetricsRegistry {
   }
 
   /**
+   * Test adding metrics with whitespace in the name
+   */
+  @Test
+  public void testMetricsRegistryIllegalMetricNames() {
+    final MetricsRegistry r = new MetricsRegistry("test");
+    // Fill up with some basics
+    r.newCounter("c1", "c1 desc", 1);
+    r.newGauge("g1", "g1 desc", 1);
+    r.newQuantiles("q1", "q1 desc", "q1 name", "q1 val type", 1);
+    // Add some illegal names
+    expectMetricsException("Metric name 'badcount 2' contains "+
+        "illegal whitespace character", new Runnable() {
+      @Override
+      public void run() { r.newCounter("badcount 2", "c2 desc", 2); }
+    });
+    expectMetricsException("Metric name 'badcount3  ' contains "+
+        "illegal whitespace character", new Runnable() {
+      @Override
+      public void run() { r.newCounter("badcount3  ", "c3 desc", 3); }
+    });
+    expectMetricsException("Metric name '  badcount4' contains "+
+        "illegal whitespace character", new Runnable() {
+      @Override
+      public void run() { r.newCounter("  badcount4", "c4 desc", 4); }
+    });
+    expectMetricsException("Metric name 'withtab5	' contains "+
+        "illegal whitespace character", new Runnable() {
+      @Override
+      public void run() { r.newCounter("withtab5	", "c5 desc", 5); }
+    });
+    expectMetricsException("Metric name 'withnewline6\n' contains "+
+        "illegal whitespace character", new Runnable() {
+      @Override
+      public void run() { r.newCounter("withnewline6\n", "c6 desc", 6); }
+    });
+    // Final validation
+    assertEquals("num metrics in registry", 3, r.metrics().size());
+  }
+
+  /**
    * Test the add by name method
    */
   @Test public void testAddByName() {
@@ -81,6 +122,7 @@ public class TestMetricsRegistry {
     });
   }
 
+  @Ignore
   private void expectMetricsException(String prefix, Runnable fun) {
     try {
       fun.run();


[08/25] hadoop git commit: HDFS-5782. Change BlockListAsLongs constructor to take Replica as parameter type instead of concrete classes Block and ReplicaInfo. Contributed by David Powell and Joe Pallas

Posted by zj...@apache.org.
HDFS-5782. Change BlockListAsLongs constructor to take Replica as parameter type instead of concrete classes Block and ReplicaInfo.  Contributed by David Powell and Joe Pallas


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

Branch: refs/heads/YARN-2928
Commit: 6464a8929a3623e49155febf2f9817253f9a1de3
Parents: 7fe0f25
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Wed Jan 14 13:46:06 2015 -0800
Committer: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Committed: Wed Jan 14 13:46:06 2015 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |  4 ++
 .../hadoop/hdfs/protocol/BlockListAsLongs.java  | 60 ++++++++++++++++----
 .../server/datanode/SimulatedFSDataset.java     |  4 +-
 .../TestBlockHasMultipleReplicasOnSameDN.java   |  2 +-
 .../server/namenode/NNThroughputBenchmark.java  |  2 +-
 5 files changed, 57 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6464a892/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 b879f62d..11d15f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -129,6 +129,10 @@ Trunk (Unreleased)
     HDFS-6609. Use DirectorySnapshottableFeature to represent a snapshottable
     directory. (Jing Zhao via wheat9)
 
+    HDFS-5782. Change BlockListAsLongs constructor to take Replica as parameter
+    type instead of concrete classes Block and ReplicaInfo.  (David Powell
+    and Joe Pallas via szetszwo)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6464a892/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
index 8a0b731..4389714 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/BlockListAsLongs.java
@@ -25,7 +25,7 @@ import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo;
+import org.apache.hadoop.hdfs.server.datanode.Replica;
 
 /**
  * This class provides an interface for accessing list of blocks that
@@ -85,8 +85,8 @@ public class BlockListAsLongs implements Iterable<Block> {
    * @param finalized - list of finalized blocks
    * @param uc - list of under construction blocks
    */
-  public BlockListAsLongs(final List<? extends Block> finalized,
-                          final List<ReplicaInfo> uc) {
+  public BlockListAsLongs(final List<? extends Replica> finalized,
+                          final List<? extends Replica> uc) {
     int finalizedSize = finalized == null ? 0 : finalized.size();
     int ucSize = uc == null ? 0 : uc.size();
     int len = HEADER_SIZE
@@ -113,8 +113,34 @@ public class BlockListAsLongs implements Iterable<Block> {
     }
   }
 
+  /**
+   * Create block report from a list of finalized blocks.  Used by
+   * NNThroughputBenchmark.
+   *
+   * @param blocks - list of finalized blocks
+   */
+  public BlockListAsLongs(final List<? extends Block> blocks) {
+    int finalizedSize = blocks == null ? 0 : blocks.size();
+    int len = HEADER_SIZE
+              + (finalizedSize + 1) * LONGS_PER_FINALIZED_BLOCK;
+
+    blockList = new long[len];
+
+    // set the header
+    blockList[0] = finalizedSize;
+    blockList[1] = 0;
+
+    // set finalized blocks
+    for (int i = 0; i < finalizedSize; i++) {
+      setBlock(i, blocks.get(i));
+    }
+
+    // set invalid delimiting block
+    setDelimitingBlock(finalizedSize);
+  }
+
   public BlockListAsLongs() {
-    this(null);
+    this((long[])null);
   }
 
   /**
@@ -279,18 +305,30 @@ public class BlockListAsLongs implements Iterable<Block> {
   /**
    * Set the indexTh block
    * @param index - the index of the block to set
-   * @param b - the block is set to the value of the this block
+   * @param r - the block is set to the value of the this Replica
    */
-  private <T extends Block> void setBlock(final int index, final T b) {
+  private void setBlock(final int index, final Replica r) {
     int pos = index2BlockId(index);
-    blockList[pos] = b.getBlockId();
-    blockList[pos + 1] = b.getNumBytes();
-    blockList[pos + 2] = b.getGenerationStamp();
+    blockList[pos] = r.getBlockId();
+    blockList[pos + 1] = r.getNumBytes();
+    blockList[pos + 2] = r.getGenerationStamp();
     if(index < getNumberOfFinalizedReplicas())
       return;
-    assert ((ReplicaInfo)b).getState() != ReplicaState.FINALIZED :
+    assert r.getState() != ReplicaState.FINALIZED :
       "Must be under-construction replica.";
-    blockList[pos + 3] = ((ReplicaInfo)b).getState().getValue();
+    blockList[pos + 3] = r.getState().getValue();
+  }
+
+  /**
+   * Set the indexTh block
+   * @param index - the index of the block to set
+   * @param b - the block is set to the value of the this Block
+   */
+  private void setBlock(final int index, final Block b) {
+    int pos = index2BlockId(index);
+    blockList[pos] = b.getBlockId();
+    blockList[pos + 1] = b.getNumBytes();
+    blockList[pos + 2] = b.getGenerationStamp();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6464a892/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
index 78eedf9..16e7a20 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java
@@ -557,12 +557,12 @@ public class SimulatedFSDataset implements FsDatasetSpi<FsVolumeSpi> {
   }
 
   synchronized BlockListAsLongs getBlockReport(String bpid) {
-    final List<Block> blocks = new ArrayList<Block>();
+    final List<Replica> blocks = new ArrayList<Replica>();
     final Map<Block, BInfo> map = blockMap.get(bpid);
     if (map != null) {
       for (BInfo b : map.values()) {
         if (b.isFinalized()) {
-          blocks.add(b.theBlock);
+          blocks.add(b);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6464a892/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 e71c0ea..1152c74 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
@@ -114,7 +114,7 @@ public class TestBlockHasMultipleReplicasOnSameDN {
     }
 
     for (int i = 0; i < cluster.getStoragesPerDatanode(); ++i) {
-      BlockListAsLongs bll = new BlockListAsLongs(blocks, null);
+      BlockListAsLongs bll = new BlockListAsLongs(blocks);
       FsVolumeSpi v = dn.getFSDataset().getVolumes().get(i);
       DatanodeStorage dns = new DatanodeStorage(v.getStorageID());
       reports[i] = new StorageBlockReport(dns, bll.getBlockListAsLongs());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6464a892/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 1fb1c1f..6abe600 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
@@ -977,7 +977,7 @@ public class NNThroughputBenchmark implements Tool {
       // fill remaining slots with blocks that do not exist
       for(int idx = blocks.size()-1; idx >= nrBlocks; idx--)
         blocks.set(idx, new Block(blocks.size() - idx, 0, 0));
-      blockReportList = new BlockListAsLongs(blocks,null).getBlockListAsLongs();
+      blockReportList = new BlockListAsLongs(blocks).getBlockListAsLongs();
     }
 
     long[] getBlockReportList() {


[10/25] hadoop git commit: Add CHANGES.txt for HADOOP-11481

Posted by zj...@apache.org.
Add CHANGES.txt for HADOOP-11481


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

Branch: refs/heads/YARN-2928
Commit: 5805dc0f0405bee418daa1c9890a669da591cf41
Parents: a5a033c
Author: Colin Patrick Mccabe <cm...@cloudera.com>
Authored: Wed Jan 14 18:06:04 2015 -0800
Committer: Colin Patrick Mccabe <cm...@cloudera.com>
Committed: Wed Jan 14 18:06:04 2015 -0800

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


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5805dc0f/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 59e6d80..9ef1fa6 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -478,6 +478,9 @@ Release 2.7.0 - UNRELEASED
     HADOOP-11465. Fix findbugs warnings in hadoop-gridmix. (Varun Saxena via
     Arpit Agarwal)
 
+    HADOOP-11481. ClassCastException while using a key created by keytool to
+    create encryption zone. (Charles Lamb via Colin P. Mccabe)
+
   OPTIMIZATIONS
 
     HADOOP-11323. WritableComparator#compare keeps reference to byte array.


[12/25] hadoop git commit: HADOOP-11483. HardLink.java should use the jdk7 createLink method

Posted by zj...@apache.org.
HADOOP-11483. HardLink.java should use the jdk7 createLink method


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

Branch: refs/heads/YARN-2928
Commit: db51548f706ccd2d0200745ab89e27610c6d10bc
Parents: ba5116e
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Jan 15 20:54:44 2015 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Jan 15 20:54:44 2015 +0900

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES.txt |   2 +
 .../java/org/apache/hadoop/fs/HardLink.java     | 313 ++-----------------
 .../java/org/apache/hadoop/fs/TestHardLink.java |  91 +-----
 3 files changed, 23 insertions(+), 383 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/db51548f/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 9ef1fa6..3f916a2 100644
--- a/hadoop-common-project/hadoop-common/CHANGES.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES.txt
@@ -385,6 +385,8 @@ Release 2.7.0 - UNRELEASED
 
   IMPROVEMENTS
 
+    HADOOP-11483. HardLink.java should use the jdk7 createLink method (aajisaka)
+
     HADOOP-11156. DelegateToFileSystem should implement
     getFsStatus(final Path f). (Zhihai Xu via wang)
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db51548f/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
index e48354d..209ba69 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java
@@ -23,13 +23,16 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.StringReader;
-import java.util.Arrays;
 
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ExitCodeException;
 import org.apache.hadoop.util.Shell.ShellCommandExecutor;
 
+import com.google.common.annotations.VisibleForTesting;
+
+import static java.nio.file.Files.createLink;
+
 /**
  * Class for creating hardlinks.
  * Supports Unix/Linux, Windows via winutils , and Mac OS X.
@@ -75,114 +78,30 @@ public class HardLink {
   
   /**
    * This abstract class bridges the OS-dependent implementations of the 
-   * needed functionality for creating hardlinks and querying link counts.
+   * needed functionality for querying link counts.
    * The particular implementation class is chosen during 
    * static initialization phase of the HardLink class.
-   * The "getter" methods construct shell command strings for various purposes.
+   * The "getter" methods construct shell command strings.
    */
   private static abstract class HardLinkCommandGetter {
-
-    /**
-     * Get the command string needed to hardlink a bunch of files from
-     * a single source directory into a target directory.  The source directory
-     * is not specified here, but the command will be executed using the source
-     * directory as the "current working directory" of the shell invocation.
-     * 
-     * @param fileBaseNames - array of path-less file names, relative
-     *            to the source directory
-     * @param linkDir - target directory where the hardlinks will be put
-     * @return - an array of Strings suitable for use as a single shell command
-     * @throws IOException - if any of the file or path names misbehave
-     */
-    abstract String[] linkMult(String[] fileBaseNames, File linkDir) 
-                          throws IOException;
-    
-    /**
-     * Get the command string needed to hardlink a single file
-     */
-    abstract String[] linkOne(File file, File linkName) throws IOException;
-    
     /**
      * Get the command string to query the hardlink count of a file
      */
     abstract String[] linkCount(File file) throws IOException;
-    
-    /**
-     * Calculate the total string length of the shell command
-     * resulting from execution of linkMult, plus the length of the
-     * source directory name (which will also be provided to the shell)
-     * 
-     * @param fileDir - source directory, parent of fileBaseNames
-     * @param fileBaseNames - array of path-less file names, relative
-     *            to the source directory
-     * @param linkDir - target directory where the hardlinks will be put
-     * @return - total data length (must not exceed maxAllowedCmdArgLength)
-     * @throws IOException
-     */
-    abstract int getLinkMultArgLength(
-                     File fileDir, String[] fileBaseNames, File linkDir) 
-                     throws IOException;
-    
-    /**
-     * Get the maximum allowed string length of a shell command on this OS,
-     * which is just the documented minimum guaranteed supported command
-     * length - aprx. 32KB for Unix, and 8KB for Windows.
-     */
-    abstract int getMaxAllowedCmdArgLength(); 
   }
   
   /**
    * Implementation of HardLinkCommandGetter class for Unix
    */
-  static class HardLinkCGUnix extends HardLinkCommandGetter {
-    private static String[] hardLinkCommand = {"ln", null, null};
-    private static String[] hardLinkMultPrefix = {"ln"};
-    private static String[] hardLinkMultSuffix = {null};
+  private static class HardLinkCGUnix extends HardLinkCommandGetter {
     private static String[] getLinkCountCommand = {"stat","-c%h", null};
-    //Unix guarantees at least 32K bytes cmd length.
-    //Subtract another 64b to allow for Java 'exec' overhead
-    private static final int maxAllowedCmdArgLength = 32*1024 - 65;
-    
     private static synchronized 
     void setLinkCountCmdTemplate(String[] template) {
       //May update this for specific unix variants, 
       //after static initialization phase
       getLinkCountCommand = template;
     }
-    
-    /*
-     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkOne(java.io.File, java.io.File)
-     */
-    @Override
-    String[] linkOne(File file, File linkName) 
-    throws IOException {
-      String[] buf = new String[hardLinkCommand.length];
-      System.arraycopy(hardLinkCommand, 0, buf, 0, hardLinkCommand.length);
-      //unix wants argument order: "ln <existing> <new>"
-      buf[1] = FileUtil.makeShellPath(file, true); 
-      buf[2] = FileUtil.makeShellPath(linkName, true);
-      return buf;
-    }
-    
-    /*
-     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkMult(java.lang.String[], java.io.File)
-     */
-    @Override
-    String[] linkMult(String[] fileBaseNames, File linkDir) 
-    throws IOException {
-      String[] buf = new String[fileBaseNames.length 
-                                + hardLinkMultPrefix.length 
-                                + hardLinkMultSuffix.length];
-      int mark=0;
-      System.arraycopy(hardLinkMultPrefix, 0, buf, mark, 
-                       hardLinkMultPrefix.length);
-      mark += hardLinkMultPrefix.length;
-      System.arraycopy(fileBaseNames, 0, buf, mark, fileBaseNames.length);
-      mark += fileBaseNames.length;
-      buf[mark] = FileUtil.makeShellPath(linkDir, true);
-      return buf;
-    }
-    
+
     /*
      * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkCount(java.io.File)
      */
@@ -195,169 +114,30 @@ public class HardLink {
       buf[getLinkCountCommand.length - 1] = FileUtil.makeShellPath(file, true);
       return buf;
     }
-    
-    /*
-     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getLinkMultArgLength(java.io.File, java.lang.String[], java.io.File)
-     */
-    @Override
-    int getLinkMultArgLength(File fileDir, String[] fileBaseNames, File linkDir) 
-    throws IOException{
-      int sum = 0;
-      for (String x : fileBaseNames) {
-        // add 1 to account for terminal null or delimiter space
-        sum += 1 + ((x == null) ? 0 : x.length());
-      }
-      sum += 2 + FileUtil.makeShellPath(fileDir, true).length()
-             + FileUtil.makeShellPath(linkDir, true).length();
-      //add the fixed overhead of the hardLinkMult prefix and suffix
-      sum += 3; //length("ln") + 1
-      return sum;
-    }
-    
-    /*
-     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getMaxAllowedCmdArgLength()
-     */
-    @Override
-    int getMaxAllowedCmdArgLength() {
-      return maxAllowedCmdArgLength;
-    }
   }
   
-  
   /**
    * Implementation of HardLinkCommandGetter class for Windows
    */
+  @VisibleForTesting
   static class HardLinkCGWin extends HardLinkCommandGetter {
-    //The Windows command getter impl class and its member fields are
-    //package-private ("default") access instead of "private" to assist 
-    //unit testing (sort of) on non-Win servers
 
-    static String CMD_EXE = "cmd.exe";
-    static String[] hardLinkCommand = {
-                        Shell.WINUTILS,"hardlink","create", null, null};
-    static String[] hardLinkMultPrefix = {
-        CMD_EXE, "/q", "/c", "for", "%f", "in", "("};
-    static String   hardLinkMultDir = "\\%f";
-    static String[] hardLinkMultSuffix = {
-        ")", "do", Shell.WINUTILS, "hardlink", "create", null,
-        "%f"};
     static String[] getLinkCountCommand = {
         Shell.WINUTILS, "hardlink", "stat", null};
-    //Windows guarantees only 8K - 1 bytes cmd length.
-    //Subtract another 64b to allow for Java 'exec' overhead
-    static final int maxAllowedCmdArgLength = 8*1024 - 65;
 
     /*
-     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkOne(java.io.File, java.io.File)
-     */
-    @Override
-    String[] linkOne(File file, File linkName) 
-    throws IOException {
-      String[] buf = new String[hardLinkCommand.length];
-      System.arraycopy(hardLinkCommand, 0, buf, 0, hardLinkCommand.length);
-      //windows wants argument order: "create <new> <existing>"
-      buf[4] = file.getCanonicalPath(); 
-      buf[3] = linkName.getCanonicalPath();
-      return buf;
-    }
-    
-    /*
-     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkMult(java.lang.String[], java.io.File)
-     */
-    @Override
-    String[] linkMult(String[] fileBaseNames, File linkDir) 
-    throws IOException {
-      String[] buf = new String[fileBaseNames.length 
-                                + hardLinkMultPrefix.length 
-                                + hardLinkMultSuffix.length];
-      String td = linkDir.getCanonicalPath() + hardLinkMultDir;
-      int mark=0;
-      System.arraycopy(hardLinkMultPrefix, 0, buf, mark, 
-                       hardLinkMultPrefix.length);
-      mark += hardLinkMultPrefix.length;
-      System.arraycopy(fileBaseNames, 0, buf, mark, fileBaseNames.length);
-      mark += fileBaseNames.length;
-      System.arraycopy(hardLinkMultSuffix, 0, buf, mark, 
-                       hardLinkMultSuffix.length);
-      mark += hardLinkMultSuffix.length;
-      buf[mark - 2] = td;
-      return buf;
-    }
-    
-    /*
      * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#linkCount(java.io.File)
      */
     @Override
-    String[] linkCount(File file) 
-    throws IOException {
+    String[] linkCount(File file) throws IOException {
       String[] buf = new String[getLinkCountCommand.length];
       System.arraycopy(getLinkCountCommand, 0, buf, 0, 
                        getLinkCountCommand.length);
       buf[getLinkCountCommand.length - 1] = file.getCanonicalPath();
       return buf;
     }
-    
-    /*
-     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getLinkMultArgLength(java.io.File, java.lang.String[], java.io.File)
-     */
-    @Override
-    int getLinkMultArgLength(File fileDir, String[] fileBaseNames, File linkDir) 
-    throws IOException {
-      int sum = 0;
-      for (String x : fileBaseNames) {
-        // add 1 to account for terminal null or delimiter space
-        sum += 1 + ((x == null) ? 0 : x.length());
-      }
-      sum += 2 + fileDir.getCanonicalPath().length() +
-               linkDir.getCanonicalPath().length();
-      //add the fixed overhead of the hardLinkMult command 
-      //(prefix, suffix, and Dir suffix)
-      sum += (CMD_EXE + " /q /c for %f in ( ) do "
-              + Shell.WINUTILS + " hardlink create \\%f %f").length();
-      return sum;
-    }
-    
-    /*
-     * @see org.apache.hadoop.fs.HardLink.HardLinkCommandGetter#getMaxAllowedCmdArgLength()
-     */
-    @Override
-    int getMaxAllowedCmdArgLength() {
-      return maxAllowedCmdArgLength;
-    }
   }
-  
-  
-  /**
-   * Calculate the nominal length of all contributors to the total 
-   * commandstring length, including fixed overhead of the OS-dependent 
-   * command.  It's protected rather than private, to assist unit testing,
-   * but real clients are not expected to need it -- see the way 
-   * createHardLinkMult() uses it internally so the user doesn't need to worry
-   * about it.
-   * 
-   * @param fileDir - source directory, parent of fileBaseNames
-   * @param fileBaseNames - array of path-less file names, relative
-   *            to the source directory
-   * @param linkDir - target directory where the hardlinks will be put
-   * @return - total data length (must not exceed maxAllowedCmdArgLength)
-   * @throws IOException
-   */
-  protected static int getLinkMultArgLength(
-          File fileDir, String[] fileBaseNames, File linkDir) 
-  throws IOException {
-    return getHardLinkCommand.getLinkMultArgLength(fileDir, 
-          fileBaseNames, linkDir);
-  }
-  
-  /**
-   * Return this private value for use by unit tests.
-   * Shell commands are not allowed to have a total string length
-   * exceeding this size.
-   */
-  protected static int getMaxAllowedCmdArgLength() {
-    return getHardLinkCommand.getMaxAllowedCmdArgLength();
-  }
-  
+
   /*
    * ****************************************************
    * Complexity is above.  User-visible functionality is below
@@ -370,7 +150,7 @@ public class HardLink {
    * @param linkName - desired target link file
    */
   public static void createHardLink(File file, File linkName) 
-  throws IOException {
+      throws IOException {
     if (file == null) {
       throw new IOException(
           "invalid arguments to createHardLink: source file is null");
@@ -379,17 +159,7 @@ public class HardLink {
       throw new IOException(
           "invalid arguments to createHardLink: link name is null");
     }
-	  // construct and execute shell command
-    String[] hardLinkCommand = getHardLinkCommand.linkOne(file, linkName);
-    ShellCommandExecutor shexec = new ShellCommandExecutor(hardLinkCommand);
-    try {
-      shexec.execute();
-    } catch (ExitCodeException e) {
-      throw new IOException("Failed to execute command " +
-          Arrays.toString(hardLinkCommand) +
-          "; command output: \"" + shexec.getOutput() + "\"" +
-          "; WrappedException: \"" + e.getMessage() + "\"");
-    }
+    createLink(linkName.toPath(), file.toPath());
   }
 
   /**
@@ -398,30 +168,10 @@ public class HardLink {
    * @param parentDir - directory containing source files
    * @param fileBaseNames - list of path-less file names, as returned by 
    *                        parentDir.list()
-   * @param linkDir - where the hardlinks should be put.  It must already exist.
-   * 
-   * If the list of files is too long (overflows maxAllowedCmdArgLength),
-   * we will automatically split it into multiple invocations of the
-   * underlying method.
+   * @param linkDir - where the hardlinks should be put. It must already exist.
    */
-  public static void createHardLinkMult(File parentDir, String[] fileBaseNames, 
+  public static void createHardLinkMult(File parentDir, String[] fileBaseNames,
       File linkDir) throws IOException {
-    //This is the public method all non-test clients are expected to use.
-    //Normal case - allow up to maxAllowedCmdArgLength characters in the cmd
-    createHardLinkMult(parentDir, fileBaseNames, linkDir, 
-                       getHardLinkCommand.getMaxAllowedCmdArgLength());
-  }
-
-  /*
-   * Implements {@link createHardLinkMult} with added variable  "maxLength",
-   * to ease unit testing of the auto-splitting feature for long lists.
-   * Likewise why it returns "callCount", the number of sub-arrays that
-   * the file list had to be split into.
-   * Non-test clients are expected to call the public method instead.
-   */
-  protected static int createHardLinkMult(File parentDir, 
-      String[] fileBaseNames, File linkDir, int maxLength) 
-  throws IOException {
     if (parentDir == null) {
       throw new IOException(
           "invalid arguments to createHardLinkMult: parent directory is null");
@@ -435,40 +185,13 @@ public class HardLink {
           "invalid arguments to createHardLinkMult: "
           + "filename list can be empty but not null");
     }
-    if (fileBaseNames.length == 0) {
-      //the OS cmds can't handle empty list of filenames, 
-      //but it's legal, so just return.
-      return 0; 
-    }
     if (!linkDir.exists()) {
       throw new FileNotFoundException(linkDir + " not found.");
     }
-
-    //if the list is too long, split into multiple invocations
-    int callCount = 0;
-    if (getLinkMultArgLength(parentDir, fileBaseNames, linkDir) > maxLength
-          && fileBaseNames.length > 1) {
-      String[] list1 = Arrays.copyOf(fileBaseNames, fileBaseNames.length/2);
-      callCount += createHardLinkMult(parentDir, list1, linkDir, maxLength);
-      String[] list2 = Arrays.copyOfRange(fileBaseNames, fileBaseNames.length/2,
-          fileBaseNames.length);
-      callCount += createHardLinkMult(parentDir, list2, linkDir, maxLength);  
-      return callCount;
-    } else {
-      callCount = 1;
-    }
-    
-    // construct and execute shell command
-    String[] hardLinkCommand = getHardLinkCommand.linkMult(fileBaseNames, 
-        linkDir);
-    ShellCommandExecutor shexec = new ShellCommandExecutor(hardLinkCommand,
-      parentDir, null, 0L);
-    try {
-      shexec.execute();
-    } catch (ExitCodeException e) {
-      throw new IOException(shexec.getOutput() + e.getMessage());
+    for (String name : fileBaseNames) {
+      createLink(linkDir.toPath().resolve(name),
+                 parentDir.toPath().resolve(name));
     }
-    return callCount;
   }
 
    /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/db51548f/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHardLink.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHardLink.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHardLink.java
index 512ba0b..c68861c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHardLink.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHardLink.java
@@ -295,91 +295,11 @@ public class TestHardLink {
     String[] emptyList = {};
     
     //test the case of empty file list
-    int callCount = createHardLinkMult(src, emptyList, tgt_mult, 
-        getMaxAllowedCmdArgLength());
-    //check no exec calls were made
-    assertEquals(0, callCount);
+    createHardLinkMult(src, emptyList, tgt_mult);
     //check nothing changed in the directory tree
     validateSetup();
   }
-  
-  /**
-   * Test createHardLinkMult(), again, this time with the "too long list" 
-   * case where the total size of the command line arguments exceed the 
-   * allowed maximum.  In this case, the list should be automatically 
-   * broken up into chunks, each chunk no larger than the max allowed.
-   * 
-   * We use an extended version of the method call, specifying the
-   * size limit explicitly, to simulate the "too long" list with a 
-   * relatively short list.
-   */
-  @Test
-  public void testCreateHardLinkMultOversizeAndEmpty() throws IOException {
-    
-    // prep long filenames - each name takes 10 chars in the arg list
-    // (9 actual chars plus terminal null or delimeter blank)
-    String name1 = "x11111111";
-    String name2 = "x22222222";
-    String name3 = "x33333333";
-    File x1_long = new File(src, name1);
-    File x2_long = new File(src, name2);
-    File x3_long = new File(src, name3);
-    //set up source files with long file names
-    x1.renameTo(x1_long);
-    x2.renameTo(x2_long);
-    x3.renameTo(x3_long);
-    //validate setup
-    assertTrue(x1_long.exists());
-    assertTrue(x2_long.exists());
-    assertTrue(x3_long.exists());
-    assertFalse(x1.exists());
-    assertFalse(x2.exists());
-    assertFalse(x3.exists());
-    
-    //prep appropriate length information to construct test case for
-    //oversize filename list
-    int callCount;
-    String[] emptyList = {};
-    String[] fileNames = src.list();
-    //get fixed size of arg list without any filenames
-    int overhead = getLinkMultArgLength(src, emptyList, tgt_mult);
-    //select a maxLength that is slightly too short to hold 3 filenames
-    int maxLength = overhead + (int)(2.5 * (float)(1 + name1.length())); 
-    
-    //now test list of three filenames when there is room for only 2.5
-    callCount = createHardLinkMult(src, fileNames, tgt_mult, maxLength);
-    //check the request was completed in exactly two "chunks"
-    assertEquals(2, callCount);
-    String[] tgt_multNames = tgt_mult.list();
-    //sort directory listings before comparsion
-    Arrays.sort(fileNames);
-    Arrays.sort(tgt_multNames);
-    //and check the results were as expected in the dir tree
-    assertArrayEquals(fileNames, tgt_multNames);
-    
-    //Test the case where maxlength is too small even for one filename.
-    //It should go ahead and try the single files.
-    
-    //Clear the test dir tree
-    FileUtil.fullyDelete(tgt_mult);
-    assertFalse(tgt_mult.exists());
-    tgt_mult.mkdirs();
-    assertTrue(tgt_mult.exists() && tgt_mult.list().length == 0);
-    //set a limit size much smaller than a single filename
-    maxLength = overhead + (int)(0.5 * (float)(1 + name1.length()));
-    //attempt the method call
-    callCount = createHardLinkMult(src, fileNames, tgt_mult, 
-        maxLength);
-    //should go ahead with each of the three single file names
-    assertEquals(3, callCount);
-    tgt_multNames = tgt_mult.list();
-    //sort directory listings before comparsion
-    Arrays.sort(fileNames);
-    Arrays.sort(tgt_multNames);
-    //and check the results were as expected in the dir tree
-    assertArrayEquals(fileNames, tgt_multNames);
-  }
-  
+
   /*
    * Assume that this test won't usually be run on a Windows box.
    * This test case allows testing of the correct syntax of the Windows
@@ -392,18 +312,13 @@ public class TestHardLink {
    */
   @Test
   public void testWindowsSyntax() {
-    class win extends HardLinkCGWin {};
+    class win extends HardLinkCGWin {}
 
     //basic checks on array lengths
-    assertEquals(5, win.hardLinkCommand.length); 
-    assertEquals(7, win.hardLinkMultPrefix.length);
-    assertEquals(7, win.hardLinkMultSuffix.length);
     assertEquals(4, win.getLinkCountCommand.length);
 
-    assertTrue(win.hardLinkMultPrefix[4].equals("%f"));
     //make sure "%f" was not munged
     assertEquals(2, ("%f").length()); 
-    assertTrue(win.hardLinkMultDir.equals("\\%f"));
     //make sure "\\%f" was munged correctly
     assertEquals(3, ("\\%f").length()); 
     assertTrue(win.getLinkCountCommand[1].equals("hardlink"));


[16/25] hadoop git commit: HDFS-7189. Add trace spans for DFSClient metadata operations. (Colin P. McCabe via yliu)

Posted by zj...@apache.org.
HDFS-7189. Add trace spans for DFSClient metadata operations. (Colin P. McCabe via yliu)


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

Branch: refs/heads/YARN-2928
Commit: c4ccbe62c0857261b197a68c1e03a02e92f21a38
Parents: 3ab3a64
Author: yliu <yl...@apache.org>
Authored: Fri Jan 16 00:23:51 2015 +0800
Committer: yliu <yl...@apache.org>
Committed: Fri Jan 16 00:23:51 2015 +0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../hadoop/hdfs/BlockStorageLocationUtil.java   |  19 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 407 ++++++++++++++++---
 .../hadoop/hdfs/DFSInotifyEventInputStream.java | 144 ++++---
 .../hdfs/protocol/CacheDirectiveIterator.java   |  10 +-
 .../hadoop/hdfs/protocol/CachePoolIterator.java |  14 +-
 .../hdfs/protocol/EncryptionZoneIterator.java   |  15 +-
 .../server/namenode/TestCacheDirectives.java    |   3 +-
 8 files changed, 478 insertions(+), 137 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4ccbe62/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 2d4c634..34ad10f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -517,6 +517,9 @@ Release 2.7.0 - UNRELEASED
 
     HDFS-7457. DatanodeID generates excessive garbage. (daryn via kihwal)
 
+    HDFS-7189. Add trace spans for DFSClient metadata operations. (Colin P.
+    McCabe via yliu)
+
   OPTIMIZATIONS
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4ccbe62/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
index ba74978..7f992c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
@@ -51,6 +51,10 @@ import org.apache.hadoop.security.token.Token;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import org.htrace.Sampler;
+import org.htrace.Span;
+import org.htrace.Trace;
+import org.htrace.TraceScope;
 
 @InterfaceAudience.Private
 @InterfaceStability.Unstable
@@ -71,7 +75,7 @@ class BlockStorageLocationUtil {
    */
   private static List<VolumeBlockLocationCallable> createVolumeBlockLocationCallables(
       Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
-      int timeout, boolean connectToDnViaHostname) {
+      int timeout, boolean connectToDnViaHostname, Span parent) {
     
     if (datanodeBlocks.isEmpty()) {
       return Lists.newArrayList();
@@ -111,7 +115,7 @@ class BlockStorageLocationUtil {
       }
       VolumeBlockLocationCallable callable = new VolumeBlockLocationCallable(
           conf, datanode, poolId, blockIds, dnTokens, timeout, 
-          connectToDnViaHostname);
+          connectToDnViaHostname, parent);
       callables.add(callable);
     }
     return callables;
@@ -131,11 +135,11 @@ class BlockStorageLocationUtil {
   static Map<DatanodeInfo, HdfsBlocksMetadata> queryDatanodesForHdfsBlocksMetadata(
       Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
       int poolsize, int timeoutMs, boolean connectToDnViaHostname)
-      throws InvalidBlockTokenException {
+        throws InvalidBlockTokenException {
 
     List<VolumeBlockLocationCallable> callables = 
         createVolumeBlockLocationCallables(conf, datanodeBlocks, timeoutMs, 
-            connectToDnViaHostname);
+            connectToDnViaHostname, Trace.currentSpan());
     
     // Use a thread pool to execute the Callables in parallel
     List<Future<HdfsBlocksMetadata>> futures = 
@@ -319,11 +323,12 @@ class BlockStorageLocationUtil {
     private final long[] blockIds;
     private final List<Token<BlockTokenIdentifier>> dnTokens;
     private final boolean connectToDnViaHostname;
+    private final Span parentSpan;
     
     VolumeBlockLocationCallable(Configuration configuration,
         DatanodeInfo datanode, String poolId, long []blockIds,
         List<Token<BlockTokenIdentifier>> dnTokens, int timeout, 
-        boolean connectToDnViaHostname) {
+        boolean connectToDnViaHostname, Span parentSpan) {
       this.configuration = configuration;
       this.timeout = timeout;
       this.datanode = datanode;
@@ -331,6 +336,7 @@ class BlockStorageLocationUtil {
       this.blockIds = blockIds;
       this.dnTokens = dnTokens;
       this.connectToDnViaHostname = connectToDnViaHostname;
+      this.parentSpan = parentSpan;
     }
     
     public DatanodeInfo getDatanodeInfo() {
@@ -342,6 +348,8 @@ class BlockStorageLocationUtil {
       HdfsBlocksMetadata metadata = null;
       // Create the RPC proxy and make the RPC
       ClientDatanodeProtocol cdp = null;
+      TraceScope scope =
+          Trace.startSpan("getHdfsBlocksMetadata", parentSpan);
       try {
         cdp = DFSUtil.createClientDatanodeProtocolProxy(datanode, configuration,
             timeout, connectToDnViaHostname);
@@ -350,6 +358,7 @@ class BlockStorageLocationUtil {
         // Bubble this up to the caller, handle with the Future
         throw e;
       } finally {
+        scope.close();
         if (cdp != null) {
           RPC.stopProxy(cdp);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4ccbe62/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 f289da7..1bb7f4a 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
@@ -998,11 +998,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#getPreferredBlockSize(String)
    */
   public long getBlockSize(String f) throws IOException {
+    TraceScope scope = getPathTraceScope("getBlockSize", f);
     try {
       return namenode.getPreferredBlockSize(f);
     } catch (IOException ie) {
       LOG.warn("Problem getting block size", ie);
       throw ie;
+    } finally {
+      scope.close();
     }
   }
 
@@ -1035,17 +1038,20 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
       throws IOException {
     assert dtService != null;
-    Token<DelegationTokenIdentifier> token =
-      namenode.getDelegationToken(renewer);
-
-    if (token != null) {
-      token.setService(this.dtService);
-      LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
-    } else {
-      LOG.info("Cannot get delegation token from " + renewer);
+    TraceScope scope = Trace.startSpan("getDelegationToken", traceSampler);
+    try {
+      Token<DelegationTokenIdentifier> token =
+        namenode.getDelegationToken(renewer);
+      if (token != null) {
+        token.setService(this.dtService);
+        LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
+      } else {
+        LOG.info("Cannot get delegation token from " + renewer);
+      }
+      return token;
+    } finally {
+      scope.close();
     }
-    return token;
-
   }
 
   /**
@@ -1216,7 +1222,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   @VisibleForTesting
   public LocatedBlocks getLocatedBlocks(String src, long start, long length)
       throws IOException {
-    return callGetBlockLocations(namenode, src, start, length);
+    TraceScope scope = getPathTraceScope("getBlockLocations", src);
+    try {
+      return callGetBlockLocations(namenode, src, start, length);
+    } finally {
+      scope.close();
+    }
   }
 
   /**
@@ -1243,12 +1254,15 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   boolean recoverLease(String src) throws IOException {
     checkOpen();
 
+    TraceScope scope = getPathTraceScope("recoverLease", src);
     try {
       return namenode.recoverLease(src, clientName);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(FileNotFoundException.class,
                                      AccessControlException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -1265,14 +1279,19 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * as the data-block the task processes. 
    */
   public BlockLocation[] getBlockLocations(String src, long start, 
-    long length) throws IOException, UnresolvedLinkException {
-    LocatedBlocks blocks = getLocatedBlocks(src, start, length);
-    BlockLocation[] locations =  DFSUtil.locatedBlocks2Locations(blocks);
-    HdfsBlockLocation[] hdfsLocations = new HdfsBlockLocation[locations.length];
-    for (int i = 0; i < locations.length; i++) {
-      hdfsLocations[i] = new HdfsBlockLocation(locations[i], blocks.get(i));
+        long length) throws IOException, UnresolvedLinkException {
+    TraceScope scope = getPathTraceScope("getBlockLocations", src);
+    try {
+      LocatedBlocks blocks = getLocatedBlocks(src, start, length);
+      BlockLocation[] locations =  DFSUtil.locatedBlocks2Locations(blocks);
+      HdfsBlockLocation[] hdfsLocations = new HdfsBlockLocation[locations.length];
+      for (int i = 0; i < locations.length; i++) {
+        hdfsLocations[i] = new HdfsBlockLocation(locations[i], blocks.get(i));
+      }
+      return hdfsLocations;
+    } finally {
+      scope.close();
     }
-    return hdfsLocations;
   }
   
   /**
@@ -1326,15 +1345,21 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
         
     // Make RPCs to the datanodes to get volume locations for its replicas
-    Map<DatanodeInfo, HdfsBlocksMetadata> metadatas = BlockStorageLocationUtil
-        .queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks,
-            getConf().getFileBlockStorageLocationsNumThreads,
-            getConf().getFileBlockStorageLocationsTimeoutMs,
-            getConf().connectToDnViaHostname);
-    
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("metadata returned: "
-          + Joiner.on("\n").withKeyValueSeparator("=").join(metadatas));
+    TraceScope scope =
+      Trace.startSpan("getBlockStorageLocations", traceSampler);
+    Map<DatanodeInfo, HdfsBlocksMetadata> metadatas;
+    try {
+      metadatas = BlockStorageLocationUtil.
+          queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks,
+              getConf().getFileBlockStorageLocationsNumThreads,
+              getConf().getFileBlockStorageLocationsTimeoutMs,
+              getConf().connectToDnViaHostname);
+      if (LOG.isTraceEnabled()) {
+        LOG.trace("metadata returned: "
+            + Joiner.on("\n").withKeyValueSeparator("=").join(metadatas));
+      }
+    } finally {
+      scope.close();
     }
     
     // Regroup the returned VolumeId metadata to again be grouped by
@@ -1354,19 +1379,24 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
       feInfo) throws IOException {
-    if (provider == null) {
-      throw new IOException("No KeyProvider is configured, cannot access" +
-          " an encrypted file");
-    }
-    EncryptedKeyVersion ekv = EncryptedKeyVersion.createForDecryption(
-        feInfo.getKeyName(), feInfo.getEzKeyVersionName(), feInfo.getIV(),
-        feInfo.getEncryptedDataEncryptionKey());
+    TraceScope scope = Trace.startSpan("decryptEDEK", traceSampler);
     try {
-      KeyProviderCryptoExtension cryptoProvider = KeyProviderCryptoExtension
-          .createKeyProviderCryptoExtension(provider);
-      return cryptoProvider.decryptEncryptedKey(ekv);
-    } catch (GeneralSecurityException e) {
-      throw new IOException(e);
+      if (provider == null) {
+        throw new IOException("No KeyProvider is configured, cannot access" +
+            " an encrypted file");
+      }
+      EncryptedKeyVersion ekv = EncryptedKeyVersion.createForDecryption(
+          feInfo.getKeyName(), feInfo.getEzKeyVersionName(), feInfo.getIV(),
+          feInfo.getEncryptedDataEncryptionKey());
+      try {
+        KeyProviderCryptoExtension cryptoProvider = KeyProviderCryptoExtension
+            .createKeyProviderCryptoExtension(provider);
+        return cryptoProvider.decryptEncryptedKey(ekv);
+      } catch (GeneralSecurityException e) {
+        throw new IOException(e);
+      }
+    } finally {
+      scope.close();
     }
   }
 
@@ -1504,7 +1534,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       throws IOException, UnresolvedLinkException {
     checkOpen();
     //    Get block info from namenode
-    return new DFSInputStream(this, src, verifyChecksum);
+    TraceScope scope = getPathTraceScope("newDFSInputStream", src);
+    try {
+      return new DFSInputStream(this, src, verifyChecksum);
+    } finally {
+      scope.close();
+    }
   }
 
   /**
@@ -1737,6 +1772,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void createSymlink(String target, String link, boolean createParent)
       throws IOException {
+    TraceScope scope = getPathTraceScope("createSymlink", target);
     try {
       FsPermission dirPerm = 
           FsPermission.getDefault().applyUMask(dfsClientConf.uMask); 
@@ -1750,6 +1786,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      DSQuotaExceededException.class,
                                      UnresolvedPathException.class,
                                      SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -1760,11 +1798,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public String getLinkTarget(String path) throws IOException { 
     checkOpen();
+    TraceScope scope = getPathTraceScope("getLinkTarget", path);
     try {
       return namenode.getLinkTarget(path);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -1824,6 +1865,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public boolean setReplication(String src, short replication)
       throws IOException {
+    TraceScope scope = getPathTraceScope("setReplication", src);
     try {
       return namenode.setReplication(src, replication);
     } catch(RemoteException re) {
@@ -1833,6 +1875,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      DSQuotaExceededException.class,
                                      UnresolvedPathException.class,
                                      SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -1843,6 +1887,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void setStoragePolicy(String src, String policyName)
       throws IOException {
+    TraceScope scope = getPathTraceScope("setStoragePolicy", src);
     try {
       namenode.setStoragePolicy(src, policyName);
     } catch (RemoteException e) {
@@ -1852,6 +1897,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                     NSQuotaExceededException.class,
                                     UnresolvedPathException.class,
                                     SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -1859,7 +1906,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @return All the existing storage policies
    */
   public BlockStoragePolicy[] getStoragePolicies() throws IOException {
-    return namenode.getStoragePolicies();
+    TraceScope scope = Trace.startSpan("getStoragePolicies", traceSampler);
+    try {
+      return namenode.getStoragePolicies();
+    } finally {
+      scope.close();
+    }
   }
 
   /**
@@ -1870,6 +1922,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   @Deprecated
   public boolean rename(String src, String dst) throws IOException {
     checkOpen();
+    TraceScope scope = getSrcDstTraceScope("rename", src, dst);
     try {
       return namenode.rename(src, dst);
     } catch(RemoteException re) {
@@ -1878,6 +1931,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      DSQuotaExceededException.class,
                                      UnresolvedPathException.class,
                                      SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -1887,12 +1942,15 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void concat(String trg, String [] srcs) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("concat", traceSampler);
     try {
       namenode.concat(trg, srcs);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      UnresolvedPathException.class,
                                      SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
   /**
@@ -1902,6 +1960,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void rename(String src, String dst, Options.Rename... options)
       throws IOException {
     checkOpen();
+    TraceScope scope = getSrcDstTraceScope("rename2", src, dst);
     try {
       namenode.rename2(src, dst, options);
     } catch(RemoteException re) {
@@ -1914,6 +1973,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      NSQuotaExceededException.class,
                                      UnresolvedPathException.class,
                                      SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -1923,11 +1984,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public boolean truncate(String src, long newLength) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("truncate", src);
     try {
       return namenode.truncate(src, newLength, clientName);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
           UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -1938,7 +2002,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   @Deprecated
   public boolean delete(String src) throws IOException {
     checkOpen();
-    return namenode.delete(src, true);
+    return delete(src, true);
   }
 
   /**
@@ -1950,6 +2014,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public boolean delete(String src, boolean recursive) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("delete", src);
     try {
       return namenode.delete(src, recursive);
     } catch(RemoteException re) {
@@ -1958,6 +2023,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      SafeModeException.class,
                                      UnresolvedPathException.class,
                                      SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
   
@@ -1987,15 +2054,17 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#getListing(String, byte[], boolean)
    */
   public DirectoryListing listPaths(String src,  byte[] startAfter,
-      boolean needLocation) 
-    throws IOException {
+      boolean needLocation) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("listPaths", src);
     try {
       return namenode.getListing(src, startAfter, needLocation);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -2009,12 +2078,15 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public HdfsFileStatus getFileInfo(String src) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("getFileInfo", src);
     try {
       return namenode.getFileInfo(src);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
   
@@ -2024,12 +2096,15 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public boolean isFileClosed(String src) throws IOException{
     checkOpen();
+    TraceScope scope = getPathTraceScope("isFileClosed", src);
     try {
       return namenode.isFileClosed(src);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
   
@@ -2043,12 +2118,15 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("getFileLinkInfo", src);
     try {
       return namenode.getFileLinkInfo(src);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      UnresolvedPathException.class);
-     }
+    } finally {
+      scope.close();
+    }
    }
   
   @InterfaceAudience.Private
@@ -2345,6 +2423,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void setPermission(String src, FsPermission permission)
       throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("setPermission", src);
     try {
       namenode.setPermission(src, permission);
     } catch(RemoteException re) {
@@ -2353,6 +2432,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      SafeModeException.class,
                                      UnresolvedPathException.class,
                                      SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -2367,6 +2448,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void setOwner(String src, String username, String groupname)
       throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("setOwner", src);
     try {
       namenode.setOwner(src, username, groupname);
     } catch(RemoteException re) {
@@ -2375,6 +2457,18 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      SafeModeException.class,
                                      UnresolvedPathException.class,
                                      SnapshotAccessControlException.class);                                   
+    } finally {
+      scope.close();
+    }
+  }
+
+  private long[] callGetStats() throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("getStats", traceSampler);
+    try {
+      return namenode.getStats();
+    } finally {
+      scope.close();
     }
   }
 
@@ -2382,7 +2476,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#getStats()
    */
   public FsStatus getDiskStatus() throws IOException {
-    long rawNums[] = namenode.getStats();
+    long rawNums[] = callGetStats();
     return new FsStatus(rawNums[0], rawNums[1], rawNums[2]);
   }
 
@@ -2392,7 +2486,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */ 
   public long getMissingBlocksCount() throws IOException {
-    return namenode.getStats()[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX];
+    return callGetStats()[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX];
   }
   
   /**
@@ -2401,7 +2495,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */
   public long getMissingReplOneBlocksCount() throws IOException {
-    return namenode.getStats()[ClientProtocol.
+    return callGetStats()[ClientProtocol.
         GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX];
   }
 
@@ -2410,7 +2504,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */ 
   public long getUnderReplicatedBlocksCount() throws IOException {
-    return namenode.getStats()[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX];
+    return callGetStats()[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX];
   }
   
   /**
@@ -2418,7 +2512,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @throws IOException
    */ 
   public long getCorruptBlocksCount() throws IOException {
-    return namenode.getStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
+    return callGetStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
   }
   
   /**
@@ -2427,18 +2521,37 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public CorruptFileBlocks listCorruptFileBlocks(String path,
                                                  String cookie)
-    throws IOException {
-    return namenode.listCorruptFileBlocks(path, cookie);
+        throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("listCorruptFileBlocks", path);
+    try {
+      return namenode.listCorruptFileBlocks(path, cookie);
+    } finally {
+      scope.close();
+    }
   }
 
   public DatanodeInfo[] datanodeReport(DatanodeReportType type)
-  throws IOException {
-    return namenode.getDatanodeReport(type);
+      throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("datanodeReport", traceSampler);
+    try {
+      return namenode.getDatanodeReport(type);
+    } finally {
+      scope.close();
+    }
   }
     
   public DatanodeStorageReport[] getDatanodeStorageReport(
       DatanodeReportType type) throws IOException {
-    return namenode.getDatanodeStorageReport(type);
+    checkOpen();
+    TraceScope scope =
+        Trace.startSpan("datanodeStorageReport", traceSampler);
+    try {
+      return namenode.getDatanodeStorageReport(type);
+    } finally {
+      scope.close();
+    }
   }
 
   /**
@@ -2462,7 +2575,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
    */
   public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
-    return namenode.setSafeMode(action, isChecked);    
+    TraceScope scope =
+        Trace.startSpan("setSafeMode", traceSampler);
+    try {
+      return namenode.setSafeMode(action, isChecked);
+    } finally {
+      scope.close();
+    }
   }
  
   /**
@@ -2476,10 +2595,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public String createSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("createSnapshot", traceSampler);
     try {
       return namenode.createSnapshot(snapshotRoot, snapshotName);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
   
@@ -2494,10 +2616,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void deleteSnapshot(String snapshotRoot, String snapshotName)
       throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("deleteSnapshot", traceSampler);
     try {
       namenode.deleteSnapshot(snapshotRoot, snapshotName);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
   
@@ -2512,10 +2638,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void renameSnapshot(String snapshotDir, String snapshotOldName,
       String snapshotNewName) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("renameSnapshot", traceSampler);
     try {
       namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
   
@@ -2528,10 +2657,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
       throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("getSnapshottableDirListing",
+        traceSampler);
     try {
       return namenode.getSnapshottableDirListing();
     } catch(RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
 
@@ -2542,10 +2675,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void allowSnapshot(String snapshotRoot) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("allowSnapshot", traceSampler);
     try {
       namenode.allowSnapshot(snapshotRoot);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
   
@@ -2556,10 +2692,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void disallowSnapshot(String snapshotRoot) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("disallowSnapshot", traceSampler);
     try {
       namenode.disallowSnapshot(snapshotRoot);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
   
@@ -2571,78 +2710,99 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public SnapshotDiffReport getSnapshotDiffReport(String snapshotDir,
       String fromSnapshot, String toSnapshot) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("getSnapshotDiffReport", traceSampler);
     try {
       return namenode.getSnapshotDiffReport(snapshotDir,
           fromSnapshot, toSnapshot);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
 
   public long addCacheDirective(
       CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("addCacheDirective", traceSampler);
     try {
       return namenode.addCacheDirective(info, flags);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
   
   public void modifyCacheDirective(
       CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("modifyCacheDirective", traceSampler);
     try {
       namenode.modifyCacheDirective(info, flags);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
 
   public void removeCacheDirective(long id)
       throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("removeCacheDirective", traceSampler);
     try {
       namenode.removeCacheDirective(id);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
   
   public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
       CacheDirectiveInfo filter) throws IOException {
-    return new CacheDirectiveIterator(namenode, filter);
+    return new CacheDirectiveIterator(namenode, filter, traceSampler);
   }
 
   public void addCachePool(CachePoolInfo info) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("addCachePool", traceSampler);
     try {
       namenode.addCachePool(info);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
 
   public void modifyCachePool(CachePoolInfo info) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("modifyCachePool", traceSampler);
     try {
       namenode.modifyCachePool(info);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
 
   public void removeCachePool(String poolName) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("removeCachePool", traceSampler);
     try {
       namenode.removeCachePool(poolName);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException();
+    } finally {
+      scope.close();
     }
   }
 
   public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
-    return new CachePoolIterator(namenode);
+    return new CachePoolIterator(namenode, traceSampler);
   }
 
   /**
@@ -2651,10 +2811,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#saveNamespace()
    */
   void saveNamespace() throws AccessControlException, IOException {
+    TraceScope scope = Trace.startSpan("saveNamespace", traceSampler);
     try {
       namenode.saveNamespace();
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -2665,10 +2828,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#rollEdits()
    */
   long rollEdits() throws AccessControlException, IOException {
+    TraceScope scope = Trace.startSpan("rollEdits", traceSampler);
     try {
       return namenode.rollEdits();
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -2684,7 +2850,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   boolean restoreFailedStorage(String arg)
       throws AccessControlException, IOException{
-    return namenode.restoreFailedStorage(arg);
+    TraceScope scope = Trace.startSpan("restoreFailedStorage", traceSampler);
+    try {
+      return namenode.restoreFailedStorage(arg);
+    } finally {
+      scope.close();
+    }
   }
 
   /**
@@ -2695,7 +2866,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#refreshNodes()
    */
   public void refreshNodes() throws IOException {
-    namenode.refreshNodes();
+    TraceScope scope = Trace.startSpan("refreshNodes", traceSampler);
+    try {
+      namenode.refreshNodes();
+    } finally {
+      scope.close();
+    }
   }
 
   /**
@@ -2704,7 +2880,12 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#metaSave(String)
    */
   public void metaSave(String pathname) throws IOException {
-    namenode.metaSave(pathname);
+    TraceScope scope = Trace.startSpan("metaSave", traceSampler);
+    try {
+      namenode.metaSave(pathname);
+    } finally {
+      scope.close();
+    }
   }
 
   /**
@@ -2716,18 +2897,33 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#setBalancerBandwidth(long)
    */
   public void setBalancerBandwidth(long bandwidth) throws IOException {
-    namenode.setBalancerBandwidth(bandwidth);
+    TraceScope scope = Trace.startSpan("setBalancerBandwidth", traceSampler);
+    try {
+      namenode.setBalancerBandwidth(bandwidth);
+    } finally {
+      scope.close();
+    }
   }
     
   /**
    * @see ClientProtocol#finalizeUpgrade()
    */
   public void finalizeUpgrade() throws IOException {
-    namenode.finalizeUpgrade();
+    TraceScope scope = Trace.startSpan("finalizeUpgrade", traceSampler);
+    try {
+      namenode.finalizeUpgrade();
+    } finally {
+      scope.close();
+    }
   }
 
   RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
-    return namenode.rollingUpgrade(action);
+    TraceScope scope = Trace.startSpan("rollingUpgrade", traceSampler);
+    try {
+      return namenode.rollingUpgrade(action);
+    } finally {
+      scope.close();
+    }
   }
 
   /**
@@ -2784,6 +2980,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     if(LOG.isDebugEnabled()) {
       LOG.debug(src + ": masked=" + absPermission);
     }
+    TraceScope scope = Trace.startSpan("mkdir", traceSampler);
     try {
       return namenode.mkdirs(src, absPermission, createParent);
     } catch(RemoteException re) {
@@ -2797,6 +2994,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      DSQuotaExceededException.class,
                                      UnresolvedPathException.class,
                                      SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
   
@@ -2807,12 +3006,15 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    * @see ClientProtocol#getContentSummary(String)
    */
   ContentSummary getContentSummary(String src) throws IOException {
+    TraceScope scope = getPathTraceScope("getContentSummary", src);
     try {
       return namenode.getContentSummary(src);
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -2832,6 +3034,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                          diskspaceQuota);
                                          
     }
+    TraceScope scope = getPathTraceScope("setQuota", src);
     try {
       namenode.setQuota(src, namespaceQuota, diskspaceQuota);
     } catch(RemoteException re) {
@@ -2841,6 +3044,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      DSQuotaExceededException.class,
                                      UnresolvedPathException.class,
                                      SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -2851,6 +3056,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
    */
   public void setTimes(String src, long mtime, long atime) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("setTimes", src);
     try {
       namenode.setTimes(src, mtime, atime);
     } catch(RemoteException re) {
@@ -2858,6 +3064,8 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      FileNotFoundException.class,
                                      UnresolvedPathException.class,
                                      SnapshotAccessControlException.class);
+    } finally {
+      scope.close();
     }
   }
 
@@ -2909,6 +3117,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public void modifyAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("modifyAclEntries", src);
     try {
       namenode.modifyAclEntries(src, aclSpec);
     } catch(RemoteException re) {
@@ -2919,12 +3128,15 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      SafeModeException.class,
                                      SnapshotAccessControlException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
   public void removeAclEntries(String src, List<AclEntry> aclSpec)
       throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("removeAclEntries", traceSampler);
     try {
       namenode.removeAclEntries(src, aclSpec);
     } catch(RemoteException re) {
@@ -2935,11 +3147,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      SafeModeException.class,
                                      SnapshotAccessControlException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
   public void removeDefaultAcl(String src) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("removeDefaultAcl", traceSampler);
     try {
       namenode.removeDefaultAcl(src);
     } catch(RemoteException re) {
@@ -2950,11 +3165,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      SafeModeException.class,
                                      SnapshotAccessControlException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
   public void removeAcl(String src) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("removeAcl", traceSampler);
     try {
       namenode.removeAcl(src);
     } catch(RemoteException re) {
@@ -2965,11 +3183,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      SafeModeException.class,
                                      SnapshotAccessControlException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
   public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
     checkOpen();
+    TraceScope scope = Trace.startSpan("setAcl", traceSampler);
     try {
       namenode.setAcl(src, aclSpec);
     } catch(RemoteException re) {
@@ -2980,11 +3201,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      SafeModeException.class,
                                      SnapshotAccessControlException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
   public AclStatus getAclStatus(String src) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("getAclStatus", src);
     try {
       return namenode.getAclStatus(src);
     } catch(RemoteException re) {
@@ -2992,41 +3216,50 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      AclException.class,
                                      FileNotFoundException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
   
   public void createEncryptionZone(String src, String keyName)
     throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("createEncryptionZone", src);
     try {
       namenode.createEncryptionZone(src, keyName);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      SafeModeException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
   public EncryptionZone getEZForPath(String src)
           throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("getEZForPath", src);
     try {
       return namenode.getEZForPath(src);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
   public RemoteIterator<EncryptionZone> listEncryptionZones()
       throws IOException {
     checkOpen();
-    return new EncryptionZoneIterator(namenode);
+    return new EncryptionZoneIterator(namenode, traceSampler);
   }
 
   public void setXAttr(String src, String name, byte[] value, 
       EnumSet<XAttrSetFlag> flag) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("setXAttr", src);
     try {
       namenode.setXAttr(src, XAttrHelper.buildXAttr(name, value), flag);
     } catch (RemoteException re) {
@@ -3036,11 +3269,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      SafeModeException.class,
                                      SnapshotAccessControlException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
   
   public byte[] getXAttr(String src, String name) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("getXAttr", src);
     try {
       final List<XAttr> xAttrs = XAttrHelper.buildXAttrAsList(name);
       final List<XAttr> result = namenode.getXAttrs(src, xAttrs);
@@ -3049,23 +3285,29 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
   
   public Map<String, byte[]> getXAttrs(String src) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("getXAttrs", src);
     try {
       return XAttrHelper.buildXAttrMap(namenode.getXAttrs(src, null));
     } catch(RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
   
   public Map<String, byte[]> getXAttrs(String src, List<String> names) 
       throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("getXAttrs", src);
     try {
       return XAttrHelper.buildXAttrMap(namenode.getXAttrs(
           src, XAttrHelper.buildXAttrs(names)));
@@ -3073,12 +3315,15 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
   
   public List<String> listXAttrs(String src)
           throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("listXAttrs", src);
     try {
       final Map<String, byte[]> xattrs =
         XAttrHelper.buildXAttrMap(namenode.listXAttrs(src));
@@ -3087,11 +3332,14 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       throw re.unwrapRemoteException(AccessControlException.class,
                                      FileNotFoundException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
   public void removeXAttr(String src, String name) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("removeXAttr", src);
     try {
       namenode.removeXAttr(src, XAttrHelper.buildXAttr(name));
     } catch(RemoteException re) {
@@ -3101,27 +3349,32 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
                                      SafeModeException.class,
                                      SnapshotAccessControlException.class,
                                      UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
   public void checkAccess(String src, FsAction mode) throws IOException {
     checkOpen();
+    TraceScope scope = getPathTraceScope("checkAccess", src);
     try {
       namenode.checkAccess(src, mode);
     } catch (RemoteException re) {
       throw re.unwrapRemoteException(AccessControlException.class,
           FileNotFoundException.class,
           UnresolvedPathException.class);
+    } finally {
+      scope.close();
     }
   }
 
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
-    return new DFSInotifyEventInputStream(namenode);
+    return new DFSInotifyEventInputStream(traceSampler, namenode);
   }
 
   public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid)
       throws IOException {
-    return new DFSInotifyEventInputStream(namenode, lastReadTxid);
+    return new DFSInotifyEventInputStream(traceSampler, namenode, lastReadTxid);
   }
 
   @Override // RemotePeerFactory
@@ -3240,4 +3493,24 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
     return scope;
   }
+
+  private static final byte[] SRC = "src".getBytes(Charset.forName("UTF-8"));
+
+  private static final byte[] DST = "dst".getBytes(Charset.forName("UTF-8"));
+
+  TraceScope getSrcDstTraceScope(String description, String src, String dst) {
+    TraceScope scope = Trace.startSpan(description, traceSampler);
+    Span span = scope.getSpan();
+    if (span != null) {
+      if (src != null) {
+        span.addKVAnnotation(SRC,
+            src.getBytes(Charset.forName("UTF-8")));
+      }
+      if (dst != null) {
+        span.addKVAnnotation(DST,
+            dst.getBytes(Charset.forName("UTF-8")));
+      }
+    }
+    return scope;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4ccbe62/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
index 83b92b9..803e4f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
@@ -26,6 +26,9 @@ import org.apache.hadoop.hdfs.inotify.EventBatchList;
 import org.apache.hadoop.hdfs.inotify.MissingEventsException;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.util.Time;
+import org.htrace.Sampler;
+import org.htrace.Trace;
+import org.htrace.TraceScope;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,6 +47,11 @@ public class DFSInotifyEventInputStream {
   public static Logger LOG = LoggerFactory.getLogger(DFSInotifyEventInputStream
       .class);
 
+  /**
+   * The trace sampler to use when making RPCs to the NameNode.
+   */
+  private final Sampler<?> traceSampler;
+
   private final ClientProtocol namenode;
   private Iterator<EventBatch> it;
   private long lastReadTxid;
@@ -59,12 +67,15 @@ public class DFSInotifyEventInputStream {
 
   private static final int INITIAL_WAIT_MS = 10;
 
-  DFSInotifyEventInputStream(ClientProtocol namenode) throws IOException {
-    this(namenode, namenode.getCurrentEditLogTxid()); // only consider new txn's
+  DFSInotifyEventInputStream(Sampler<?> traceSampler, ClientProtocol namenode)
+        throws IOException {
+    // Only consider new transaction IDs.
+    this(traceSampler, namenode, namenode.getCurrentEditLogTxid());
   }
 
-  DFSInotifyEventInputStream(ClientProtocol namenode, long lastReadTxid)
-      throws IOException {
+  DFSInotifyEventInputStream(Sampler traceSampler, ClientProtocol namenode,
+        long lastReadTxid) throws IOException {
+    this.traceSampler = traceSampler;
     this.namenode = namenode;
     this.it = Iterators.emptyIterator();
     this.lastReadTxid = lastReadTxid;
@@ -87,39 +98,45 @@ public class DFSInotifyEventInputStream {
    * The next available batch of events will be returned.
    */
   public EventBatch poll() throws IOException, MissingEventsException {
-    // need to keep retrying until the NN sends us the latest committed txid
-    if (lastReadTxid == -1) {
-      LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN");
-      lastReadTxid = namenode.getCurrentEditLogTxid();
-      return null;
-    }
-    if (!it.hasNext()) {
-      EventBatchList el = namenode.getEditsFromTxid(lastReadTxid + 1);
-      if (el.getLastTxid() != -1) {
-        // we only want to set syncTxid when we were actually able to read some
-        // edits on the NN -- otherwise it will seem like edits are being
-        // generated faster than we can read them when the problem is really
-        // that we are temporarily unable to read edits
-        syncTxid = el.getSyncTxid();
-        it = el.getBatches().iterator();
-        long formerLastReadTxid = lastReadTxid;
-        lastReadTxid = el.getLastTxid();
-        if (el.getFirstTxid() != formerLastReadTxid + 1) {
-          throw new MissingEventsException(formerLastReadTxid + 1,
-              el.getFirstTxid());
+    TraceScope scope =
+        Trace.startSpan("inotifyPoll", traceSampler);
+    try {
+      // need to keep retrying until the NN sends us the latest committed txid
+      if (lastReadTxid == -1) {
+        LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN");
+        lastReadTxid = namenode.getCurrentEditLogTxid();
+        return null;
+      }
+      if (!it.hasNext()) {
+        EventBatchList el = namenode.getEditsFromTxid(lastReadTxid + 1);
+        if (el.getLastTxid() != -1) {
+          // we only want to set syncTxid when we were actually able to read some
+          // edits on the NN -- otherwise it will seem like edits are being
+          // generated faster than we can read them when the problem is really
+          // that we are temporarily unable to read edits
+          syncTxid = el.getSyncTxid();
+          it = el.getBatches().iterator();
+          long formerLastReadTxid = lastReadTxid;
+          lastReadTxid = el.getLastTxid();
+          if (el.getFirstTxid() != formerLastReadTxid + 1) {
+            throw new MissingEventsException(formerLastReadTxid + 1,
+                el.getFirstTxid());
+          }
+        } else {
+          LOG.debug("poll(): read no edits from the NN when requesting edits " +
+            "after txid {}", lastReadTxid);
+          return null;
         }
+      }
+
+      if (it.hasNext()) { // can be empty if el.getLastTxid != -1 but none of the
+        // newly seen edit log ops actually got converted to events
+        return it.next();
       } else {
-        LOG.debug("poll(): read no edits from the NN when requesting edits " +
-          "after txid {}", lastReadTxid);
         return null;
       }
-    }
-
-    if (it.hasNext()) { // can be empty if el.getLastTxid != -1 but none of the
-      // newly seen edit log ops actually got converted to events
-      return it.next();
-    } else {
-      return null;
+    } finally {
+      scope.close();
     }
   }
 
@@ -163,25 +180,29 @@ public class DFSInotifyEventInputStream {
    */
   public EventBatch poll(long time, TimeUnit tu) throws IOException,
       InterruptedException, MissingEventsException {
-    long initialTime = Time.monotonicNow();
-    long totalWait = TimeUnit.MILLISECONDS.convert(time, tu);
-    long nextWait = INITIAL_WAIT_MS;
+    TraceScope scope = Trace.startSpan("inotifyPollWithTimeout", traceSampler);
     EventBatch next = null;
-    while ((next = poll()) == null) {
-      long timeLeft = totalWait - (Time.monotonicNow() - initialTime);
-      if (timeLeft <= 0) {
-        LOG.debug("timed poll(): timed out");
-        break;
-      } else if (timeLeft < nextWait * 2) {
-        nextWait = timeLeft;
-      } else {
-        nextWait *= 2;
+    try {
+      long initialTime = Time.monotonicNow();
+      long totalWait = TimeUnit.MILLISECONDS.convert(time, tu);
+      long nextWait = INITIAL_WAIT_MS;
+      while ((next = poll()) == null) {
+        long timeLeft = totalWait - (Time.monotonicNow() - initialTime);
+        if (timeLeft <= 0) {
+          LOG.debug("timed poll(): timed out");
+          break;
+        } else if (timeLeft < nextWait * 2) {
+          nextWait = timeLeft;
+        } else {
+          nextWait *= 2;
+        }
+        LOG.debug("timed poll(): poll() returned null, sleeping for {} ms",
+            nextWait);
+        Thread.sleep(nextWait);
       }
-      LOG.debug("timed poll(): poll() returned null, sleeping for {} ms",
-          nextWait);
-      Thread.sleep(nextWait);
+    } finally {
+      scope.close();
     }
-
     return next;
   }
 
@@ -196,18 +217,23 @@ public class DFSInotifyEventInputStream {
    */
   public EventBatch take() throws IOException, InterruptedException,
       MissingEventsException {
+    TraceScope scope = Trace.startSpan("inotifyTake", traceSampler);
     EventBatch next = null;
-    int nextWaitMin = INITIAL_WAIT_MS;
-    while ((next = poll()) == null) {
-      // sleep for a random period between nextWaitMin and nextWaitMin * 2
-      // to avoid stampedes at the NN if there are multiple clients
-      int sleepTime = nextWaitMin + rng.nextInt(nextWaitMin);
-      LOG.debug("take(): poll() returned null, sleeping for {} ms", sleepTime);
-      Thread.sleep(sleepTime);
-      // the maximum sleep is 2 minutes
-      nextWaitMin = Math.min(60000, nextWaitMin * 2);
+    try {
+      int nextWaitMin = INITIAL_WAIT_MS;
+      while ((next = poll()) == null) {
+        // sleep for a random period between nextWaitMin and nextWaitMin * 2
+        // to avoid stampedes at the NN if there are multiple clients
+        int sleepTime = nextWaitMin + rng.nextInt(nextWaitMin);
+        LOG.debug("take(): poll() returned null, sleeping for {} ms", sleepTime);
+        Thread.sleep(sleepTime);
+        // the maximum sleep is 2 minutes
+        nextWaitMin = Math.min(60000, nextWaitMin * 2);
+      }
+    } finally {
+      scope.close();
     }
 
     return next;
   }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4ccbe62/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
index 676106d..d28b771 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
@@ -27,6 +27,9 @@ import org.apache.hadoop.fs.InvalidRequestException;
 import org.apache.hadoop.ipc.RemoteException;
 
 import com.google.common.base.Preconditions;
+import org.htrace.Sampler;
+import org.htrace.Trace;
+import org.htrace.TraceScope;
 
 /**
  * CacheDirectiveIterator is a remote iterator that iterates cache directives.
@@ -39,12 +42,14 @@ public class CacheDirectiveIterator
 
   private CacheDirectiveInfo filter;
   private final ClientProtocol namenode;
+  private final Sampler<?> traceSampler;
 
   public CacheDirectiveIterator(ClientProtocol namenode,
-      CacheDirectiveInfo filter) {
+      CacheDirectiveInfo filter, Sampler<?> traceSampler) {
     super(0L);
     this.namenode = namenode;
     this.filter = filter;
+    this.traceSampler = traceSampler;
   }
 
   private static CacheDirectiveInfo removeIdFromFilter(CacheDirectiveInfo filter) {
@@ -89,6 +94,7 @@ public class CacheDirectiveIterator
   public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
       throws IOException {
     BatchedEntries<CacheDirectiveEntry> entries = null;
+    TraceScope scope = Trace.startSpan("listCacheDirectives", traceSampler);
     try {
       entries = namenode.listCacheDirectives(prevKey, filter);
     } catch (IOException e) {
@@ -110,6 +116,8 @@ public class CacheDirectiveIterator
             "Did not find requested id " + id);
       }
       throw e;
+    } finally {
+      scope.close();
     }
     Preconditions.checkNotNull(entries);
     return entries;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4ccbe62/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
index 44d6b45..1f17c8e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
@@ -23,6 +23,9 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.htrace.Sampler;
+import org.htrace.Trace;
+import org.htrace.TraceScope;
 
 /**
  * CachePoolIterator is a remote iterator that iterates cache pools.
@@ -34,16 +37,23 @@ public class CachePoolIterator
     extends BatchedRemoteIterator<String, CachePoolEntry> {
 
   private final ClientProtocol namenode;
+  private final Sampler traceSampler;
 
-  public CachePoolIterator(ClientProtocol namenode) {
+  public CachePoolIterator(ClientProtocol namenode, Sampler traceSampler) {
     super("");
     this.namenode = namenode;
+    this.traceSampler = traceSampler;
   }
 
   @Override
   public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
       throws IOException {
-    return namenode.listCachePools(prevKey);
+    TraceScope scope = Trace.startSpan("listCachePools", traceSampler);
+    try {
+      return namenode.listCachePools(prevKey);
+    } finally {
+      scope.close();
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4ccbe62/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
index b8c21b0..8a648e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
@@ -23,6 +23,9 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.htrace.Sampler;
+import org.htrace.Trace;
+import org.htrace.TraceScope;
 
 /**
  * EncryptionZoneIterator is a remote iterator that iterates over encryption
@@ -34,16 +37,24 @@ public class EncryptionZoneIterator
     extends BatchedRemoteIterator<Long, EncryptionZone> {
 
   private final ClientProtocol namenode;
+  private final Sampler<?> traceSampler;
 
-  public EncryptionZoneIterator(ClientProtocol namenode) {
+  public EncryptionZoneIterator(ClientProtocol namenode,
+                                Sampler<?> traceSampler) {
     super(Long.valueOf(0));
     this.namenode = namenode;
+    this.traceSampler = traceSampler;
   }
 
   @Override
   public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
       throws IOException {
-    return namenode.listEncryptionZones(prevId);
+    TraceScope scope = Trace.startSpan("listEncryptionZones", traceSampler);
+    try {
+      return namenode.listEncryptionZones(prevId);
+    } finally {
+      scope.close();
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c4ccbe62/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
index 9307692..70410ce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCacheDirectives.java
@@ -88,6 +88,7 @@ import org.apache.hadoop.util.GSet;
 import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.Logger;
+import org.htrace.Sampler;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -908,7 +909,7 @@ public class TestCacheDirectives {
 
     // Uncache and check each path in sequence
     RemoteIterator<CacheDirectiveEntry> entries =
-      new CacheDirectiveIterator(nnRpc, null);
+      new CacheDirectiveIterator(nnRpc, null, Sampler.NEVER);
     for (int i=0; i<numFiles; i++) {
       CacheDirectiveEntry entry = entries.next();
       nnRpc.removeCacheDirective(entry.getInfo().getId());


[11/25] hadoop git commit: YARN-2217. [YARN-1492] Shared cache client side changes. (Chris Trezzo via kasha)

Posted by zj...@apache.org.
YARN-2217. [YARN-1492] Shared cache client side changes. (Chris Trezzo via kasha)


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

Branch: refs/heads/YARN-2928
Commit: ba5116ec8e0c075096c6f84a8c8a1c6ce8297cf2
Parents: 5805dc0
Author: Karthik Kambatla <ka...@apache.org>
Authored: Thu Jan 15 11:13:47 2015 +0530
Committer: Karthik Kambatla <ka...@apache.org>
Committed: Thu Jan 15 14:28:44 2015 +0530

----------------------------------------------------------------------
 hadoop-yarn-project/CHANGES.txt                 |   3 +
 .../yarn/client/api/SharedCacheClient.java      | 108 ++++++++++++
 .../client/api/impl/SharedCacheClientImpl.java  | 166 ++++++++++++++++++
 .../api/impl/TestSharedCacheClientImpl.java     | 170 +++++++++++++++++++
 4 files changed, 447 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ba5116ec/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index a398347..5716b50 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -75,6 +75,9 @@ Release 2.7.0 - UNRELEASED
     YARN-2427. Added the API of moving apps between queues in RM web services.
     (Varun Vasudev via zjshen)
 
+    YARN-2217. [YARN-1492] Shared cache client side changes. 
+    (Chris Trezzo via kasha)
+
   IMPROVEMENTS
 
     YARN-2950. Change message to mandate, not suggest JS requirement on UI.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ba5116ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/SharedCacheClient.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/SharedCacheClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/SharedCacheClient.java
new file mode 100644
index 0000000..7cbe0e1
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/SharedCacheClient.java
@@ -0,0 +1,108 @@
+/**
+ * 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.client.api;
+
+
+import java.io.IOException;
+
+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.fs.Path;
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.client.api.impl.SharedCacheClientImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * This is the client for YARN's shared cache.
+ */
+@Public
+@Unstable
+public abstract class SharedCacheClient extends AbstractService {
+
+  @Public
+  public static SharedCacheClient createSharedCacheClient() {
+    SharedCacheClient client = new SharedCacheClientImpl();
+    return client;
+  }
+
+  @Private
+  public SharedCacheClient(String name) {
+    super(name);
+  }
+
+  /**
+   * <p>
+   * The method to claim a resource with the <code>SharedCacheManager.</code>
+   * The client uses a checksum to identify the resource and an
+   * {@link ApplicationId} to identify which application will be using the
+   * resource.
+   * </p>
+   * 
+   * <p>
+   * The <code>SharedCacheManager</code> responds with whether or not the
+   * resource exists in the cache. If the resource exists, a <code>Path</code>
+   * to the resource in the shared cache is returned. If the resource does not
+   * exist, null is returned instead.
+   * </p>
+   * 
+   * @param applicationId ApplicationId of the application using the resource
+   * @param resourceKey the key (i.e. checksum) that identifies the resource
+   * @return Path to the resource, or null if it does not exist
+   */
+  @Public
+  @Unstable
+  public abstract Path use(ApplicationId applicationId, String resourceKey)
+      throws YarnException;
+
+  /**
+   * <p>
+   * The method to release a resource with the <code>SharedCacheManager.</code>
+   * This method is called once an application is no longer using a claimed
+   * resource in the shared cache. The client uses a checksum to identify the
+   * resource and an {@link ApplicationId} to identify which application is
+   * releasing the resource.
+   * </p>
+   * 
+   * <p>
+   * Note: This method is an optimization and the client is not required to call
+   * it for correctness.
+   * </p>
+   * 
+   * @param applicationId ApplicationId of the application releasing the
+   *          resource
+   * @param resourceKey the key (i.e. checksum) that identifies the resource
+   */
+  @Public
+  @Unstable
+  public abstract void release(ApplicationId applicationId, String resourceKey)
+      throws YarnException;
+
+  /**
+   * A convenience method to calculate the checksum of a specified file.
+   * 
+   * @param sourceFile A path to the input file
+   * @return A hex string containing the checksum digest
+   * @throws IOException
+   */
+  @Public
+  @Unstable
+  public abstract String getFileChecksum(Path sourceFile) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ba5116ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/SharedCacheClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/SharedCacheClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/SharedCacheClientImpl.java
new file mode 100644
index 0000000..0a61ee0
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/SharedCacheClientImpl.java
@@ -0,0 +1,166 @@
+/**
+ * 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.client.api.impl;
+
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.yarn.api.ClientSCMProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.ReleaseSharedCacheResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UseSharedCacheResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UseSharedCacheResourceResponse;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.client.api.SharedCacheClient;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.sharedcache.SharedCacheChecksum;
+import org.apache.hadoop.yarn.sharedcache.SharedCacheChecksumFactory;
+import org.apache.hadoop.yarn.util.Records;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * An implementation of the SharedCacheClient API.
+ */
+@Private
+@Unstable
+public class SharedCacheClientImpl extends SharedCacheClient {
+  private static final Log LOG = LogFactory
+      .getLog(SharedCacheClientImpl.class);
+
+  private ClientSCMProtocol scmClient;
+  private InetSocketAddress scmAddress;
+  private Configuration conf;
+  private SharedCacheChecksum checksum;
+
+  public SharedCacheClientImpl() {
+    super(SharedCacheClientImpl.class.getName());
+  }
+
+  private static InetSocketAddress getScmAddress(Configuration conf) {
+    return conf.getSocketAddr(YarnConfiguration.SCM_CLIENT_SERVER_ADDRESS,
+        YarnConfiguration.DEFAULT_SCM_CLIENT_SERVER_ADDRESS,
+        YarnConfiguration.DEFAULT_SCM_CLIENT_SERVER_PORT);
+  }
+
+  @Override
+  protected void serviceInit(Configuration conf) throws Exception {
+    if (this.scmAddress == null) {
+      this.scmAddress = getScmAddress(conf);
+    }
+    this.conf = conf;
+    this.checksum = SharedCacheChecksumFactory.getChecksum(conf);
+    super.serviceInit(conf);
+  }
+
+  @Override
+  protected void serviceStart() throws Exception {
+    this.scmClient = createClientProxy();
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Connecting to Shared Cache Manager at " + this.scmAddress);
+    }
+    super.serviceStart();
+  }
+
+  @Override
+  protected void serviceStop() throws Exception {
+    stopClientProxy();
+    super.serviceStop();
+  }
+
+  @VisibleForTesting
+  protected ClientSCMProtocol createClientProxy() {
+    YarnRPC rpc = YarnRPC.create(getConfig());
+    return (ClientSCMProtocol) rpc.getProxy(ClientSCMProtocol.class,
+        this.scmAddress, getConfig());
+  }
+
+  @VisibleForTesting
+  protected void stopClientProxy() {
+    if (this.scmClient != null) {
+      RPC.stopProxy(this.scmClient);
+      this.scmClient = null;
+    }
+  }
+
+  @Override
+  public Path use(ApplicationId applicationId, String resourceKey)
+      throws YarnException {
+    Path resourcePath = null;
+    UseSharedCacheResourceRequest request = Records.newRecord(
+        UseSharedCacheResourceRequest.class);
+    request.setAppId(applicationId);
+    request.setResourceKey(resourceKey);
+    try {
+      UseSharedCacheResourceResponse response = this.scmClient.use(request);
+      if (response != null && response.getPath() != null) {
+        resourcePath = new Path(response.getPath());
+      }
+    } catch (Exception e) {
+      // Just catching IOException isn't enough.
+      // RPC call can throw ConnectionException.
+      // We don't handle different exceptions separately at this point.
+      throw new YarnException(e);
+    }
+    return resourcePath;
+  }
+
+  @Override
+  public void release(ApplicationId applicationId, String resourceKey)
+      throws YarnException {
+    ReleaseSharedCacheResourceRequest request = Records.newRecord(
+        ReleaseSharedCacheResourceRequest.class);
+    request.setAppId(applicationId);
+    request.setResourceKey(resourceKey);
+    try {
+      // We do not care about the response because it is empty.
+      this.scmClient.release(request);
+    } catch (Exception e) {
+      // Just catching IOException isn't enough.
+      // RPC call can throw ConnectionException.
+      throw new YarnException(e);
+    }
+  }
+
+  @Override
+  public String getFileChecksum(Path sourceFile)
+      throws IOException {
+    FileSystem fs = sourceFile.getFileSystem(this.conf);
+    FSDataInputStream in = null;
+    try {
+      in = fs.open(sourceFile);
+      return this.checksum.computeChecksum(in);
+    } finally {
+      if (in != null) {
+        in.close();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ba5116ec/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestSharedCacheClientImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestSharedCacheClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestSharedCacheClientImpl.java
new file mode 100644
index 0000000..3985e54
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestSharedCacheClientImpl.java
@@ -0,0 +1,170 @@
+/**
+ * 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.client.api.impl;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Matchers.isA;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.ClientSCMProtocol;
+import org.apache.hadoop.yarn.api.protocolrecords.ReleaseSharedCacheResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UseSharedCacheResourceRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.UseSharedCacheResourceResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.UseSharedCacheResourceResponsePBImpl;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestSharedCacheClientImpl {
+
+  private static final Log LOG = LogFactory
+      .getLog(TestSharedCacheClientImpl.class);
+
+  public static SharedCacheClientImpl client;
+  public static ClientSCMProtocol cProtocol;
+  private static Path TEST_ROOT_DIR;
+  private static FileSystem localFs;
+  private static String input = "This is a test file.";
+  private static String inputChecksumSHA256 =
+      "f29bc64a9d3732b4b9035125fdb3285f5b6455778edca72414671e0ca3b2e0de";
+
+  @BeforeClass
+  public static void beforeClass() throws IOException {
+    localFs = FileSystem.getLocal(new Configuration());
+    TEST_ROOT_DIR =
+        new Path("target", TestSharedCacheClientImpl.class.getName()
+            + "-tmpDir").makeQualified(localFs.getUri(),
+            localFs.getWorkingDirectory());
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    try {
+      if (localFs != null) {
+        localFs.close();
+      }
+    } catch (IOException ioe) {
+      LOG.info("IO exception in closing file system)");
+      ioe.printStackTrace();
+    }
+  }
+
+  @Before
+  public void setup() {
+    cProtocol = mock(ClientSCMProtocol.class);
+    client = new SharedCacheClientImpl() {
+      @Override
+      protected ClientSCMProtocol createClientProxy() {
+        return cProtocol;
+      }
+
+      @Override
+      protected void stopClientProxy() {
+        // do nothing because it is mocked
+      }
+    };
+    client.init(new Configuration());
+    client.start();
+  }
+
+  @After
+  public void cleanup() {
+    if (client != null) {
+      client.stop();
+      client = null;
+    }
+  }
+
+  @Test
+  public void testUse() throws Exception {
+    Path file = new Path("viewfs://test/path");
+    UseSharedCacheResourceResponse response =
+        new UseSharedCacheResourceResponsePBImpl();
+    response.setPath(file.toString());
+    when(cProtocol.use(isA(UseSharedCacheResourceRequest.class))).thenReturn(
+        response);
+    Path newPath = client.use(mock(ApplicationId.class), "key");
+    assertEquals(file, newPath);
+  }
+
+  @Test(expected = YarnException.class)
+  public void testUseError() throws Exception {
+    String message = "Mock IOExcepiton!";
+    when(cProtocol.use(isA(UseSharedCacheResourceRequest.class))).thenThrow(
+        new IOException(message));
+    client.use(mock(ApplicationId.class), "key");
+  }
+
+  @Test
+  public void testRelease() throws Exception {
+    // Release does not care about the return value because it is empty
+    when(cProtocol.release(isA(ReleaseSharedCacheResourceRequest.class)))
+        .thenReturn(null);
+    client.release(mock(ApplicationId.class), "key");
+  }
+
+  @Test(expected = YarnException.class)
+  public void testReleaseError() throws Exception {
+    String message = "Mock IOExcepiton!";
+    when(cProtocol.release(isA(ReleaseSharedCacheResourceRequest.class)))
+        .thenThrow(new IOException(message));
+    client.release(mock(ApplicationId.class), "key");
+  }
+
+  @Test
+  public void testChecksum() throws Exception {
+    String filename = "test1.txt";
+    Path file = makeFile(filename);
+    assertEquals(inputChecksumSHA256, client.getFileChecksum(file));
+  }
+
+  @Test(expected = FileNotFoundException.class)
+  public void testNonexistantFileChecksum() throws Exception {
+    Path file = new Path(TEST_ROOT_DIR, "non-existant-file");
+    client.getFileChecksum(file);
+  }
+
+  private Path makeFile(String filename) throws Exception {
+    Path file = new Path(TEST_ROOT_DIR, filename);
+    DataOutputStream out = null;
+    try {
+      out = localFs.create(file);
+      out.write(input.getBytes("UTF-8"));
+    } finally {
+      if(out != null) {
+        out.close();
+      }
+    }
+    return file;
+  }
+}